iit2009060 commented on code in PR #21535:
URL: https://github.com/apache/kafka/pull/21535#discussion_r2877046656


##########
streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalStateManagerImpl.java:
##########
@@ -311,33 +316,84 @@ private void reprocessState(final List<TopicPartition> 
topicPartitions,
                             record.headers());
                     globalProcessorContext.setRecordContext(recordContext);
 
-                    try {
-                        if (record.key() != null) {
-                            source.process(new Record(
+                    if (record.key() != null) {
+                        // Deserialization phase
+                        final Record<?, ?> deserializedRecord;
+                        try {
+                            deserializedRecord = new Record<>(
                                 
reprocessFactory.keyDeserializer().deserialize(record.topic(), record.key()),
                                 
reprocessFactory.valueDeserializer().deserialize(record.topic(), 
record.value()),
                                 record.timestamp(),
-                                record.headers()));
+                                record.headers());
+                        } catch (final Exception deserializationException) {
+                            // while Java distinguishes checked vs unchecked 
exceptions, other languages
+                            // like Scala or Kotlin do not, and thus we need 
to catch `Exception`
+                            // (instead of `RuntimeException`) to work well 
with those languages
+                            handleDeserializationFailure(
+                                deserializationExceptionHandler,
+                                globalProcessorContext,
+                                deserializationException,
+                                record,
+                                log,
+                                droppedRecordsSensor(
+                                    Thread.currentThread().getName(),
+                                    globalProcessorContext.taskId().toString(),
+                                    globalProcessorContext.metrics()
+                                ),
+                                null
+                            );
+                            continue; // Skip this record
+                        }
+
+                        // Processing phase
+                        try {
+                            @SuppressWarnings("unchecked")
+                            final Processor<Object, Object, Object, Object> 
typedSource = 
+                                (Processor<Object, Object, Object, Object>) 
source;
+                            @SuppressWarnings("unchecked")
+                            final Record<Object, Object> typedRecord = 
(Record<Object, Object>) deserializedRecord;
+                            typedSource.process(typedRecord);
                             restoreCount++;
                             batchRestoreCount++;
+                        } catch (final Exception processingException) {
+                            // while Java distinguishes checked vs unchecked 
exceptions, other languages
+                            // like Scala or Kotlin do not, and thus we need 
to catch `Exception`
+                            // (instead of `RuntimeException`) to work well 
with those languages
+                            if (processingExceptionHandler != null) {
+                                final ErrorHandlerContext errorHandlerContext 
= new DefaultErrorHandlerContext(
+                                    globalProcessorContext,
+                                    record.topic(),
+                                    record.partition(),
+                                    record.offset(),
+                                    record.headers(),
+                                    storeName,
+                                    globalProcessorContext.taskId(),
+                                    record.timestamp(),
+                                    record.key(),
+                                    record.value()
+                                );
+                                final ProcessingExceptionHandler.Response 
response = 
+                                    processingExceptionHandler.handleError(
+                                        errorHandlerContext,
+                                        deserializedRecord,
+                                        processingException
+                                    );
+                                
+                                if (response.result() == 
ProcessingExceptionHandler.Result.FAIL) {
+                                    log.error("Processing exception handler 
chose to fail for record at offset {}", record.offset());
+                                    throw processingException;
+                                }
+                                // RESUME - log and continue
+                                log.warn("Processing exception handler chose 
to resume for record at offset {}", record.offset(), processingException);
+                                droppedRecordsSensor(

Review Comment:
   done



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

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to