From ab8fc86382fc9f5a8a0f696a91c66806d51c05a5 Mon Sep 17 00:00:00 2001 From: MayureshGharat Date: Wed, 18 May 2016 01:49:54 +0100 Subject: [PATCH] KAFKA-3393; Updated the docs to reflect the deprecation of block.on.buffer.full and usage of max.block.ms Author: MayureshGharat Reviewers: Grant Henke , Ismael Juma Closes #1060 from MayureshGharat/KAFKA-3393 --- .../apache/kafka/clients/producer/KafkaProducer.java | 6 +++--- .../apache/kafka/clients/producer/ProducerConfig.java | 11 +++++------ docs/upgrade.html | 1 + 3 files changed, 9 insertions(+), 9 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java b/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java index 5b05272ab22..fd3eb096c1c 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java @@ -119,8 +119,8 @@ import org.slf4j.LoggerFactory; *

* The buffer.memory 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. For uses where you want to avoid any blocking you can set block.on.buffer.full=false which - * will cause the send call to result in an exception. + * exhausted additional send calls will block. The threshold for time to block is determined by max.block.ms after which it throws + * a TimeoutException. *

* The key.serializer and value.serializer instruct how to turn the key and value objects the user provides with * their ProducerRecord into bytes. You can use the included {@link org.apache.kafka.common.serialization.ByteArraySerializer} or @@ -420,7 +420,7 @@ public class KafkaProducer implements Producer { * * @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 BufferExhaustedException If block.on.buffer.full=false and the buffer is full. + * @throws TimeoutException if the time taken for fetching metadata or allocating memory for the record has surpassed max.block.ms. * */ @Override diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java b/clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java index 5b7a296a903..4ed083b2e79 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java @@ -139,14 +139,14 @@ public class ProducerConfig extends AbstractConfig { /** block.on.buffer.full */ /** - * @deprecated This config will be removed in a future release. Also, the {@link #METADATA_FETCH_TIMEOUT_CONFIG} is no longer honored when this property is set to true. + * @deprecated This config will be removed in a future release. Please use {@link #MAX_BLOCK_MS_CONFIG}. */ @Deprecated public static final String BLOCK_ON_BUFFER_FULL_CONFIG = "block.on.buffer.full"; private static final String BLOCK_ON_BUFFER_FULL_DOC = "When our memory buffer is exhausted we must either stop accepting new records (block) or throw errors. " - + "By default this setting is false and the producer will throw a BufferExhaustedException if a record is sent and the buffer space is full. " - + "However in some scenarios getting an error is not desirable and it is better to block. Setting this to true will accomplish that." - + "If this property is set to true, parameter " + METADATA_FETCH_TIMEOUT_CONFIG + " is not longer honored." + + "By default this setting is false and the producer will no longer throw a BufferExhaustException but instead will use the {@link #MAX_BLOCK_MS_CONFIG} " + + "value to block, after which it will throw a TimeoutException. Setting this property to true will set the " + MAX_BLOCK_MS_CONFIG + " to Long.MAX_VALUE." + + "Also if this property is set to true, parameter " + METADATA_FETCH_TIMEOUT_CONFIG + " is not longer honored." + "

" + "This parameter is deprecated and will be removed in a future release. " + "Parameter " + MAX_BLOCK_MS_CONFIG + " should be used instead."; @@ -154,8 +154,7 @@ public class ProducerConfig extends AbstractConfig { /** buffer.memory */ 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 either block or throw an exception based " - + "on the preference specified by " + BLOCK_ON_BUFFER_FULL_CONFIG + ". " + + "sent faster than they can be delivered to the server the producer will block for " + MAX_BLOCK_MS_CONFIG + " after which it will throw an exception." + "

" + "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 " diff --git a/docs/upgrade.html b/docs/upgrade.html index d09b9d76bea..dec0808c2e3 100644 --- a/docs/upgrade.html +++ b/docs/upgrade.html @@ -164,6 +164,7 @@ work with 0.10.0.x brokers. Therefore, 0.9.0.0 clients should be upgraded to 0.9

  • Altering topic configuration from the kafka-topics.sh script (kafka.admin.TopicCommand) has been deprecated. Going forward, please use the kafka-configs.sh script (kafka.admin.ConfigCommand) for this functionality.
  • The kafka-consumer-offset-checker.sh (kafka.tools.ConsumerOffsetChecker) has been deprecated. Going forward, please use kafka-consumer-groups.sh (kafka.admin.ConsumerGroupCommand) for this functionality.
  • The kafka.tools.ProducerPerformance class has been deprecated. Going forward, please use org.apache.kafka.tools.ProducerPerformance for this functionality (kafka-producer-perf-test.sh will also be changed to use the new class).
  • +
  • The producer config block.on.buffer.full has been deprecated and will be removed in future release. Currently its default value has been changed to false. The KafkaProducer will no longer throw BufferExhaustedException but instead will use max.block.ms value to block, after which it will throw a TimeoutException. If block.on.buffer.full property is set to true explicitly, it will set the max.block.ms to Long.MAX_VALUE and metadata.fetch.timeout.ms will not be honoured
  • Upgrading from 0.8.1 to 0.8.2