Interface DeserializationExceptionHandler
- All Superinterfaces:
org.apache.kafka.common.Configurable
- All Known Implementing Classes:
LogAndContinueExceptionHandler,LogAndFailExceptionHandler
public interface DeserializationExceptionHandler
extends org.apache.kafka.common.Configurable
Interface that specifies how an exception from source node deserialization
(e.g., reading from Kafka) should be handled.
-
Nested Class Summary
Nested ClassesModifier and TypeInterfaceDescriptionstatic enumDeprecated.static classRepresents the result of handling a deserialization exception.static enumEnumeration that describes the response from the exception handler. -
Method Summary
Modifier and TypeMethodDescriptionhandle(ErrorHandlerContext context, org.apache.kafka.clients.consumer.ConsumerRecord<byte[], byte[]> record, Exception exception) Deprecated.handle(ProcessorContext context, org.apache.kafka.clients.consumer.ConsumerRecord<byte[], byte[]> record, Exception exception) Deprecated.Since 3.9.handleError(ErrorHandlerContext context, org.apache.kafka.clients.consumer.ConsumerRecord<byte[], byte[]> record, Exception exception) Inspects a record and the exception received during deserialization.Methods inherited from interface org.apache.kafka.common.Configurable
configure
-
Method Details
-
handle
@Deprecated default DeserializationExceptionHandler.DeserializationHandlerResponse handle(ProcessorContext context, org.apache.kafka.clients.consumer.ConsumerRecord<byte[], byte[]> record, Exception exception) Deprecated.Since 3.9. Usehandle(ErrorHandlerContext, ConsumerRecord, Exception)instead.Inspect a record and the exception received.Note, that the passed in
ProcessorContextonly allows to access metadata like the task ID. However, it cannot be used to emit records viaProcessorContext.forward(Object, Object); callingforward()(and some other methods) would result in a runtime exception.- Parameters:
context- Processor context.record- Record that failed deserialization.exception- The actual exception.- Returns:
- Whether to continue or stop processing.
-
handle
@Deprecated default DeserializationExceptionHandler.DeserializationHandlerResponse handle(ErrorHandlerContext context, org.apache.kafka.clients.consumer.ConsumerRecord<byte[], byte[]> record, Exception exception) Deprecated.Inspect a record and the exception received.- Parameters:
context- Error handler context.record- Record that failed deserialization.exception- The actual exception.- Returns:
- Whether to continue or stop processing.
-
handleError
default DeserializationExceptionHandler.Response handleError(ErrorHandlerContext context, org.apache.kafka.clients.consumer.ConsumerRecord<byte[], byte[]> record, Exception exception) Inspects a record and the exception received during deserialization.- Parameters:
context- Error handler context.record- Record that failed deserialization.exception- The actual exception.- Returns:
- a
DeserializationExceptionHandler.Responseobject
-
handleError(ErrorHandlerContext, ConsumerRecord, Exception)instead.