diff --git a/streams/src/main/java/org/apache/kafka/streams/errors/DeserializationExceptionHandler.java b/streams/src/main/java/org/apache/kafka/streams/errors/DeserializationExceptionHandler.java index 0d64611de678d..9e7ee94f46d65 100644 --- a/streams/src/main/java/org/apache/kafka/streams/errors/DeserializationExceptionHandler.java +++ b/streams/src/main/java/org/apache/kafka/streams/errors/DeserializationExceptionHandler.java @@ -37,7 +37,7 @@ public interface DeserializationExceptionHandler extends Configurable { * @param context processor context * @param record record that failed deserialization * @param exception the actual exception - * @deprecated Since 3.9. Use Please {@link #handle(ErrorHandlerContext, ConsumerRecord, Exception)} + * @deprecated Since 3.9. Use {@link #handle(ErrorHandlerContext, ConsumerRecord, Exception)} */ @Deprecated default DeserializationHandlerResponse handle(final ProcessorContext context, diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalStateManagerImpl.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalStateManagerImpl.java index 6b7214a9ed185..e5daa151f4ede 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalStateManagerImpl.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalStateManagerImpl.java @@ -330,7 +330,8 @@ private void reprocessState(final List topicPartitions, Thread.currentThread().getName(), globalProcessorContext.taskId().toString(), globalProcessorContext.metrics() - ) + ), + null ); } } diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordDeserializer.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordDeserializer.java index 8ee2dc014ebc6..11775eb38e468 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordDeserializer.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordDeserializer.java @@ -75,15 +75,6 @@ ConsumerRecord deserialize(final ProcessorContext processo } } - public static void handleDeserializationFailure(final DeserializationExceptionHandler deserializationExceptionHandler, - final ProcessorContext processorContext, - final Exception deserializationException, - final ConsumerRecord rawRecord, - final Logger log, - final Sensor droppedRecordsSensor) { - handleDeserializationFailure(deserializationExceptionHandler, processorContext, deserializationException, rawRecord, log, droppedRecordsSensor, null); - } - public static void handleDeserializationFailure(final DeserializationExceptionHandler deserializationExceptionHandler, final ProcessorContext processorContext, final Exception deserializationException,