mirror of https://github.com/apache/kafka.git
KAFKA-16372 Fix producer doc discrepancy with the exception behavior (#15574)
Currently, Producer.send doc is inconsistent with actual exception behavior - TimeoutException: This won't be thrown from send on buffer-full or metadata-missing actually. Instead, it will returned as failed future. - AuthenticationException/AuthorizationException: These exceptions are also won't be thrown. Returned with failed future actually. Fixed Callback javadoc and ProducerConfig doc as well. Reviewers: Luke Chen <showuon@gmail.com>, Andrew Schofield <aschofield@confluent.io>
This commit is contained in:
parent
5d81fe20c8
commit
17846fe743
|
@ -42,6 +42,8 @@ public interface Callback {
|
|||
* <li>{@link org.apache.kafka.common.errors.UnknownServerException UnknownServerException}
|
||||
* <li>{@link org.apache.kafka.common.errors.UnknownProducerIdException UnknownProducerIdException}
|
||||
* <li>{@link org.apache.kafka.common.errors.InvalidProducerEpochException InvalidProducerEpochException}
|
||||
* <li>{@link org.apache.kafka.common.errors.AuthenticationException AuthenticationException}
|
||||
* <li>{@link org.apache.kafka.common.errors.AuthorizationException AuthorizationException}
|
||||
* </ul>
|
||||
* Retriable exceptions (transient, may be covered by increasing #.retries):
|
||||
* <ul>
|
||||
|
@ -52,6 +54,7 @@ public interface Callback {
|
|||
* <li>{@link org.apache.kafka.common.errors.OffsetOutOfRangeException OffsetOutOfRangeException}
|
||||
* <li>{@link org.apache.kafka.common.errors.TimeoutException TimeoutException}
|
||||
* <li>{@link org.apache.kafka.common.errors.UnknownTopicOrPartitionException UnknownTopicOrPartitionException}
|
||||
* <li>{@link org.apache.kafka.clients.producer.BufferExhaustedException BufferExhaustedException}
|
||||
* </ul>
|
||||
*/
|
||||
void onCompletion(RecordMetadata metadata, Exception exception);
|
||||
|
|
|
@ -149,8 +149,8 @@ import java.util.concurrent.atomic.AtomicReference;
|
|||
* <p>
|
||||
* The <code>buffer.memory</code> controls the total amount of memory available to the producer for buffering. If records
|
||||
* are sent faster than they can be transmitted to the server then this buffer space will be exhausted. When the buffer space is
|
||||
* exhausted additional send calls will block. The threshold for time to block is determined by <code>max.block.ms</code> after which it throws
|
||||
* a TimeoutException.
|
||||
* exhausted additional send calls will block. The threshold for time to block is determined by <code>max.block.ms</code> after which it returns
|
||||
* a failed future with BufferExhaustedException.
|
||||
* <p>
|
||||
* The <code>key.serializer</code> and <code>value.serializer</code> instruct how to turn the key and value objects the user provides with
|
||||
* their <code>ProducerRecord</code> into bytes. You can use the included {@link org.apache.kafka.common.serialization.ByteArraySerializer} or
|
||||
|
@ -926,14 +926,10 @@ public class KafkaProducer<K, V> implements Producer<K, V> {
|
|||
* @param callback A user-supplied callback to execute when the record has been acknowledged by the server (null
|
||||
* indicates no callback)
|
||||
*
|
||||
* @throws AuthenticationException if authentication fails. See the exception for more details
|
||||
* @throws AuthorizationException fatal error indicating that the producer is not allowed to write
|
||||
* @throws IllegalStateException if a transactional.id has been configured and no transaction has been started, or
|
||||
* when send is invoked after producer has been closed.
|
||||
* @throws InterruptException If the thread is interrupted while blocked
|
||||
* @throws SerializationException If the key or value are not valid objects given the configured serializers
|
||||
* @throws TimeoutException If the record could not be appended to the send buffer due to memory unavailable
|
||||
* or missing metadata within {@code max.block.ms}.
|
||||
* @throws KafkaException If a Kafka related error occurs that does not belong to the public API exceptions.
|
||||
*/
|
||||
@Override
|
||||
|
|
|
@ -209,7 +209,7 @@ public class ProducerConfig extends AbstractConfig {
|
|||
/** <code>buffer.memory</code> */
|
||||
public static final String BUFFER_MEMORY_CONFIG = "buffer.memory";
|
||||
private static final String BUFFER_MEMORY_DOC = "The total bytes of memory the producer can use to buffer records waiting to be sent to the server. If records are "
|
||||
+ "sent faster than they can be delivered to the server the producer will block for <code>" + MAX_BLOCK_MS_CONFIG + "</code> after which it will throw an exception."
|
||||
+ "sent faster than they can be delivered to the server the producer will block for <code>" + MAX_BLOCK_MS_CONFIG + "</code> after which it will fail with an exception."
|
||||
+ "<p>"
|
||||
+ "This setting should correspond roughly to the total memory the producer will use, but is not a hard bound since "
|
||||
+ "not all memory the producer uses is used for buffering. Some additional memory will be used for compression (if "
|
||||
|
|
Loading…
Reference in New Issue