sebastienviale commented on code in PR #17942:
URL: https://github.com/apache/kafka/pull/17942#discussion_r2210292952


##########
streams/src/main/java/org/apache/kafka/streams/errors/DefaultProductionExceptionHandler.java:
##########
@@ -18,38 +18,42 @@
 
 import org.apache.kafka.clients.producer.ProducerRecord;
 import org.apache.kafka.common.errors.RetriableException;
+import org.apache.kafka.streams.StreamsConfig;
 
 import java.util.Map;
 
+import static 
org.apache.kafka.streams.errors.internals.ExceptionHandlerUtils.maybeBuildDeadLetterQueueRecords;
+
 /**
  * {@code ProductionExceptionHandler} that always instructs streams to fail 
when an exception
  * happens while attempting to produce result records.
  */
 public class DefaultProductionExceptionHandler implements 
ProductionExceptionHandler {
-    /**
-     * @deprecated Since 3.9. Use {@link #handle(ErrorHandlerContext, 
ProducerRecord, Exception)} instead.
-     */
-    @SuppressWarnings("deprecation")
-    @Deprecated
+
+    private String deadLetterQueueTopic = null;
+
     @Override
-    public ProductionExceptionHandlerResponse handle(final 
ProducerRecord<byte[], byte[]> record,
-                                                     final Exception 
exception) {
+    public Response handleError(final ErrorHandlerContext context,
+                                final ProducerRecord<byte[], byte[]> record,
+                                final Exception exception) {
         return exception instanceof RetriableException ?
-            ProductionExceptionHandlerResponse.RETRY :
-            ProductionExceptionHandlerResponse.FAIL;
+            Response.retry() :
+            
Response.fail(maybeBuildDeadLetterQueueRecords(deadLetterQueueTopic, 
context.sourceRawKey(), context.sourceRawKey(), context, exception));
     }
 
+    @SuppressWarnings("rawtypes")
     @Override
-    public ProductionExceptionHandlerResponse handle(final ErrorHandlerContext 
context,
-                                                     final 
ProducerRecord<byte[], byte[]> record,
-                                                     final Exception 
exception) {
-        return exception instanceof RetriableException ?
-            ProductionExceptionHandlerResponse.RETRY :
-            ProductionExceptionHandlerResponse.FAIL;
+    public Response handleSerializationError(final ErrorHandlerContext context,
+                                             final ProducerRecord record,

Review Comment:
   I would say we do not know the ProducerRecord key and value types because it 
is not serialized yet.



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