Skip to content

Commit

Permalink
update batch.size doc (apache#11160)
Browse files Browse the repository at this point in the history
We didn't mention anything about the linger.ms and upper the bound batch size concept in the batch.size, which will make users believe we'll send every batch with this batch.size setting value.

Reviewers: Boyang Chen <[email protected]>
  • Loading branch information
showuon authored Aug 1, 2021
1 parent 4efd9bf commit 6fbea57
Showing 1 changed file with 7 additions and 2 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -81,7 +81,12 @@ public class ProducerConfig extends AbstractConfig {
+ "<p>"
+ "A small batch size will make batching less common and may reduce throughput (a batch size of zero will disable "
+ "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 of the specified batch size in anticipation of additional records."
+ "<p>"
+ "Note: This setting gives the upper bound of the batch size to be sent. If we have fewer than this many bytes accumulated "
+ "for this partition, we will 'linger' for the <code>linger.ms</code> time waiting for more records to show up. "
+ "This <code>linger.ms</code> setting defaults to 0, which means we'll immediately send out a record even the accumulated "
+ "batch size is under this <code>batch.size</code> setting.";

/** <code>acks</code> */
public static final String ACKS_CONFIG = "acks";
Expand All @@ -106,7 +111,7 @@ public class ProducerConfig extends AbstractConfig {
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&mdash;that is, rather than immediately sending out a record the producer will wait for up to "
+ "of artificial delay&mdash;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 <code>" + BATCH_SIZE_CONFIG + "</code> worth of records for a partition it will be sent immediately regardless of this "
Expand Down

0 comments on commit 6fbea57

Please sign in to comment.