Interface ProductionExceptionHandler
- All Superinterfaces:
org.apache.kafka.common.Configurable
- All Known Implementing Classes:
DefaultProductionExceptionHandler
public interface ProductionExceptionHandler
extends org.apache.kafka.common.Configurable
Interface that specifies how an exception when attempting to produce a result to
Kafka should be handled.
-
Nested Class Summary
Nested ClassesModifier and TypeInterfaceDescriptionstatic enumDeprecated.static classRepresents the result of handling a production exception.static enumEnumeration that describes the response from the exception handler.static enum -
Method Summary
Modifier and TypeMethodDescriptionDeprecated.Since 3.9.handle(ErrorHandlerContext context, org.apache.kafka.clients.producer.ProducerRecord<byte[], byte[]> record, Exception exception) Deprecated.handleError(ErrorHandlerContext context, org.apache.kafka.clients.producer.ProducerRecord<byte[], byte[]> record, Exception exception) Inspect a record that we attempted to produce, and the exception that resulted from attempting to produce it and determine to continue or stop processing.handleSerializationError(ErrorHandlerContext context, org.apache.kafka.clients.producer.ProducerRecord record, Exception exception, ProductionExceptionHandler.SerializationExceptionOrigin origin) Handles serialization exception and determine if the process should continue.handleSerializationException(org.apache.kafka.clients.producer.ProducerRecord record, Exception exception) Deprecated.Since 3.9.handleSerializationException(ErrorHandlerContext context, org.apache.kafka.clients.producer.ProducerRecord record, Exception exception, ProductionExceptionHandler.SerializationExceptionOrigin origin) Methods inherited from interface org.apache.kafka.common.Configurable
configure
-
Method Details
-
handle
@Deprecated default ProductionExceptionHandler.ProductionExceptionHandlerResponse handle(org.apache.kafka.clients.producer.ProducerRecord<byte[], byte[]> record, Exception exception) Deprecated.Since 3.9. Usehandle(ErrorHandlerContext, ProducerRecord, Exception)instead.Inspect a record that we attempted to produce, and the exception that resulted from attempting to produce it and determine to continue or stop processing.- Parameters:
record- The record that failed to produce.exception- The exception that occurred during production.- Returns:
- Whether to continue or stop processing, or retry the failed operation.
-
handle
@Deprecated default ProductionExceptionHandler.ProductionExceptionHandlerResponse handle(ErrorHandlerContext context, org.apache.kafka.clients.producer.ProducerRecord<byte[], byte[]> record, Exception exception) Deprecated.Inspect a record that we attempted to produce, and the exception that resulted from attempting to produce it and determine to continue or stop processing.- Parameters:
context- The error handler context metadata.record- The record that failed to produce.exception- The exception that occurred during production.- Returns:
- Whether to continue or stop processing, or retry the failed operation.
-
handleError
default ProductionExceptionHandler.Response handleError(ErrorHandlerContext context, org.apache.kafka.clients.producer.ProducerRecord<byte[], byte[]> record, Exception exception) Inspect a record that we attempted to produce, and the exception that resulted from attempting to produce it and determine to continue or stop processing.- Parameters:
context- The error handler context metadata.record- The record that failed to produce.exception- The exception that occurred during production.- Returns:
- a
ProductionExceptionHandler.Responseobject
-
handleSerializationException
@Deprecated default ProductionExceptionHandler.ProductionExceptionHandlerResponse handleSerializationException(org.apache.kafka.clients.producer.ProducerRecord record, Exception exception) Deprecated.Handles serialization exception and determine if the process should continue. The default implementation is to fail the process.- Parameters:
record- The record that failed to serialize.exception- The exception that occurred during serialization.- Returns:
- Whether to continue or stop processing, or retry the failed operation.
-
handleSerializationException
@Deprecated default ProductionExceptionHandler.ProductionExceptionHandlerResponse handleSerializationException(ErrorHandlerContext context, org.apache.kafka.clients.producer.ProducerRecord record, Exception exception, ProductionExceptionHandler.SerializationExceptionOrigin origin) Deprecated.Handles serialization exception and determine if the process should continue. The default implementation is to fail the process.- Parameters:
context- The error handler context metadata.record- The record that failed to serialize.exception- The exception that occurred during serialization.origin- The origin of the serialization exception.- Returns:
- Whether to continue or stop processing, or retry the failed operation.
-
handleSerializationError
default ProductionExceptionHandler.Response handleSerializationError(ErrorHandlerContext context, org.apache.kafka.clients.producer.ProducerRecord record, Exception exception, ProductionExceptionHandler.SerializationExceptionOrigin origin) Handles serialization exception and determine if the process should continue. The default implementation is to fail the process.- Parameters:
context- The error handler context metadata.record- The record that failed to serialize.exception- The exception that occurred during serialization.origin- The origin of the serialization exception.- Returns:
- a
ProductionExceptionHandler.Responseobject
-