From 1ff61a8c613f60322ae8f2898ffb079daf6f71a0 Mon Sep 17 00:00:00 2001 From: mjsax Date: Tue, 23 Feb 2016 15:13:38 +0100 Subject: [PATCH] [MINOR] Fixed documenation of parameter "block.on.buffer.full"; default value is "false" and not "true" - extended documentation according to reviewer comment. - minor improvements of other parmeters: * use variables instead of hard coded parameter names * replace JavaDoc with proper HTML markup --- .../clients/producer/ProducerConfig.java | 62 +++++++++++-------- 1 file changed, 37 insertions(+), 25 deletions(-) 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 ee2b142a70fde..5b7a296a903b1 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 @@ -59,7 +59,8 @@ public class ProducerConfig extends AbstractConfig { /** batch.size */ public static final String BATCH_SIZE_CONFIG = "batch.size"; - private static final String BATCH_SIZE_DOC = "The producer will attempt to batch records together into fewer requests whenever multiple records are being sent" + " to the same partition. This helps performance on both the client and the server. This configuration controls the " + private static final String BATCH_SIZE_DOC = "The producer will attempt to batch records together into fewer requests whenever multiple records are being sent" + + " to the same partition. This helps performance on both the client and the server. This configuration controls the " + "default batch size in bytes. " + "

" + "No attempt will be made to batch records larger than this size. " @@ -70,15 +71,6 @@ public class ProducerConfig extends AbstractConfig { + "batching entirely). A very large batch size may use memory a bit more wastefully as we will always allocate a " + "buffer of the specified batch size in anticipation of additional records."; - /** 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. " - + "

" - + "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 " - + "compression is enabled) as well as for maintaining in-flight requests."; - /** acks */ public static final String ACKS_CONFIG = "acks"; private static final String ACKS_DOC = "The number of acknowledgments the producer requires the leader to have received before considering a request complete. This controls the " @@ -103,20 +95,22 @@ public class ProducerConfig extends AbstractConfig { */ @Deprecated public static final String TIMEOUT_CONFIG = "timeout.ms"; - private static final String TIMEOUT_DOC = "The configuration controls the maximum amount of time the server will wait for acknowledgments from followers to " + "meet the acknowledgment requirements the producer has specified with the acks configuration. If the " + private static final String TIMEOUT_DOC = "The configuration controls the maximum amount of time the server will wait for acknowledgments from followers to " + + "meet the acknowledgment requirements the producer has specified with the acks configuration. If the " + "requested number of acknowledgments are not met when the timeout elapses an error will be returned. This timeout " + "is measured on the server side and does not include the network latency of the request."; /** linger.ms */ public static final String LINGER_MS_CONFIG = "linger.ms"; - private static final String LINGER_MS_DOC = "The producer groups together any records that arrive in between request transmissions into a single batched request. " + "Normally this occurs only under load when records arrive faster than they can be sent out. However in some circumstances the client may want to " + private static final String LINGER_MS_DOC = "The producer groups together any records that arrive in between request transmissions into a single batched request. " + + "Normally this occurs only under load when records arrive faster than they can be sent out. However in some circumstances the client may want to " + "reduce the number of requests even under moderate load. This setting accomplishes this by adding a small amount " + "of artificial delay—that is, rather than immediately sending out a record the producer will wait for up to " + "the given delay to allow other records to be sent so that the sends can be batched together. This can be thought " + "of as analogous to Nagle's algorithm in TCP. This setting gives the upper bound on the delay for batching: once " - + "we get batch.size worth of records for a partition it will be sent immediately regardless of this " + + "we get " + BATCH_SIZE_CONFIG + " worth of records for a partition it will be sent immediately regardless of this " + "setting, however if we have fewer than this many bytes accumulated for this partition we will 'linger' for the " - + "specified time waiting for more records to show up. This setting defaults to 0 (i.e. no delay). Setting linger.ms=5, " + + "specified time waiting for more records to show up. This setting defaults to 0 (i.e. no delay). Setting " + LINGER_MS_CONFIG + "=5, " + "for example, would have the effect of reducing the number of requests sent but would add up to 5ms of latency to records sent in the absense of load."; /** client.id */ @@ -130,24 +124,47 @@ public class ProducerConfig extends AbstractConfig { /** max.request.size */ public static final String MAX_REQUEST_SIZE_CONFIG = "max.request.size"; - private static final String MAX_REQUEST_SIZE_DOC = "The maximum size of a request in bytes. This is also effectively a cap on the maximum record size. Note that the server " + "has its own cap on record size which may be different from this. This setting will limit the number of record " + private static final String MAX_REQUEST_SIZE_DOC = "The maximum size of a request in bytes. This is also effectively a cap on the maximum record size. Note that the server " + + "has its own cap on record size which may be different from this. This setting will limit the number of record " + "batches the producer will send in a single request to avoid sending huge requests."; /** reconnect.backoff.ms */ public static final String RECONNECT_BACKOFF_MS_CONFIG = CommonClientConfigs.RECONNECT_BACKOFF_MS_CONFIG; + /** max.block.ms */ + public static final String MAX_BLOCK_MS_CONFIG = "max.block.ms"; + private static final String MAX_BLOCK_MS_DOC = "The configuration controls how long KafkaProducer.send() and KafkaProducer.partitionsFor() will block." + + "These methods can be blocked either because the buffer is full or metadata unavailable." + + "Blocking in the user-supplied serializers or partitioner will not be counted against this timeout."; + /** 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 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 true and we block, however in some scenarios blocking is not desirable and it is better to " - + "immediately give an error. Setting this to false will accomplish that: the producer will throw a BufferExhaustedException if a record is sent and the buffer space is 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." + + "

" + + "This parameter is deprecated and will be removed in a future release. " + + "Parameter " + MAX_BLOCK_MS_CONFIG + " should be used instead."; + + /** 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 + ". " + + "

" + + "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 " + + "compression is enabled) as well as for maintaining in-flight requests."; /** retries */ public static final String RETRIES_CONFIG = "retries"; - private static final String RETRIES_DOC = "Setting a value greater than zero will cause the client to resend any record whose send fails with a potentially transient error." + " Note that this retry is no different than if the client resent the record upon receiving the " + private static final String RETRIES_DOC = "Setting a value greater than zero will cause the client to resend any record whose send fails with a potentially transient error." + + " Note that this retry is no different than if the client resent the record upon receiving the " + "error. Allowing retries will potentially change the ordering of records because if two records are " + "sent to a single partition, and the first fails and is retried but the second succeeds, then the second record " + "may appear first."; @@ -157,7 +174,8 @@ public class ProducerConfig extends AbstractConfig { /** compression.type */ public static final String COMPRESSION_TYPE_CONFIG = "compression.type"; - private static final String COMPRESSION_TYPE_DOC = "The compression type for all data generated by the producer. The default is none (i.e. no compression). Valid " + " values are none, gzip, snappy, or lz4. " + private static final String COMPRESSION_TYPE_DOC = "The compression type for all data generated by the producer. The default is none (i.e. no compression). Valid " + + " values are none, gzip, snappy, or lz4. " + "Compression is of full batches of data, so the efficacy of batching will also impact the compression ratio (more batching means better compression)."; /** metrics.sample.window.ms */ @@ -190,12 +208,6 @@ public class ProducerConfig extends AbstractConfig { public static final String PARTITIONER_CLASS_CONFIG = "partitioner.class"; private static final String PARTITIONER_CLASS_DOC = "Partitioner class that implements the Partitioner interface."; - /** max.block.ms */ - public static final String MAX_BLOCK_MS_CONFIG = "max.block.ms"; - private static final String MAX_BLOCK_MS_DOC = "The configuration controls how long {@link KafkaProducer#send()} and {@link KafkaProducer#partitionsFor} will block." - + "These methods can be blocked either because the buffer is full or metadata unavailable." - + "Blocking in the user-supplied serializers or partitioner will not be counted against this timeout."; - /** request.timeout.ms */ public static final String REQUEST_TIMEOUT_MS_CONFIG = CommonClientConfigs.REQUEST_TIMEOUT_MS_CONFIG; private static final String REQUEST_TIMEOUT_MS_DOC = CommonClientConfigs.REQUEST_TIMEOUT_MS_DOC;