kafka-dev mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From Jay Kreps <jay.kr...@gmail.com>
Subject config documentation
Date Thu, 29 Aug 2013 05:06:32 GMT
I took a pass through the configuration docs (
http://localhost/documentation.html#configuration) and tried to make them
more understandable. We really could do better at this. If we are going to
take the time to make a configuration we should bother to explain exactly
what it does and tell people how to use it properly. It would be really
great if others could do a pass through and sanity check the docs with an
eye towards the user.

Here is an example of bad configuration documentation:

 property: fetch.purgatory.purge.interval.requests
 default: 10000
 description: The purge interval (in number of requests) of the fetch
request purgatory

What is a request purgatory? What is a purge interval? What does it mean to
purge a request purgatory? How should I set this value?

Or

property: consumer.id
default: null
description: Generated automatically if not set.

This kind of fails to answer the obvious question: what is a consumer id
and why might I want to set one?

I think this actually accounts for a fairly high percentage of the support
load as well as just scaring people away.

Here is a complete diff of my changes:

jkreps-mn:kafka-site jkreps$ svn diff
Index: 08/configuration.html
===================================================================
--- 08/configuration.html (revision 1518456)
+++ 08/configuration.html (working copy)
@@ -1,3 +1,7 @@
+Kafka uses the <a href="http://en.wikipedia.org/wiki/.properties">property
file format</a> for configuration. These can be supplied either from a file
or programmatically.
+<p>
+Some configurations have both a default global setting as well as a
topic-level overrides. The topic level properties have the format of csv
(e.g., "xyz.per.topic=topic1:value1,topic2:value2") and they override the
default value for the specified topics.
+
 <h3><a id="brokerconfigs">3.1 Broker Configs</a></h3>
 The essential configurations are the following:
 <ul>
@@ -6,8 +10,6 @@
  <li><code>zookeeper.connect</code>
 </ul>

-Note that some configurations have both a default global setting as well
as a topic level setting. The topic level properties have the format of csv
(e.g., "topic1:value1,topic2:value2") and they override the values in the
global setting for those specified topics.
-
 <table class="data-table">
 <tbody><tr>
       <th>Property</th>
@@ -17,14 +19,20 @@
     <tr>
       <td>broker.id</td>
       <td></td>
-      <td>Each broker is uniquely identified by a non-negative integer id.
This id serves as the brokers "name", and allows the broker to be moved to
a different host/port without confusing consumers.</td>
+      <td>Each broker is uniquely identified by a non-negative integer id.
This id serves as the brokers "name", and allows the broker to be moved to
a different host/port without confusing consumers. You can choose any
number you like so long as it is unique.
+ </td>
     </tr>
     <tr>
       <td>log.dirs</td>
       <td nowrap>/tmp/kafka-logs</td>
-      <td>The directories in which the log data is kept</td>
+      <td>A comma-separated list of one or more directories in which Kafka
data is stored. Each new partition that is created will be placed in the
directory which currently has the fewest partitions.</td>
     </tr>
     <tr>
+      <td>port</td>
+      <td>6667</td>
+      <td>The port on which the server accepts client connections.</td>
+    </tr>
+    <tr>
       <td>zookeeper.connect</td>
       <td>null</td>
       <td>Specifies the zookeeper connection string in the form
<code>hostname:port</code>, where hostname and port are the host and port
for a node in your zookeeper cluster. To allow connecting through other
zookeeper nodes when that host is down you can also specify multiple hosts
in the form <code>hostname1:port1,hostname2:port2,hostname3:port3</code>.
@@ -34,140 +42,135 @@
     <tr>
       <td>message.max.bytes</td>
       <td>1000000</td>
-      <td>The maximum size of a message that the server can receive</td>
+      <td>The maximum size of a message that the server can receive. It is
important that this property be in sync with the maximum fetch size your
consumers use or else an unruly consumer will be able to publish messages
too large for consumers to consume.</td>
     </tr>
     <tr>
       <td>num.network.threads</td>
       <td>3</td>
-      <td>The number of network threads that the server uses for handling
network requests</td>
+      <td>The number of network threads that the server uses for handling
network requests. You probably don't need to change this.</td>
     </tr>
     <tr>
       <td>num.io.threads</td>
       <td>8</td>
-      <td>The number of io threads that the server uses for carrying out
network requests</td>
+      <td>The number of I/O threads that the server uses for executing
requests. You should have at least as many threads as you have disks.</td>
     </tr>
     <tr>
       <td>queued.max.requests</td>
       <td>500</td>
-      <td>The number of queued requests allowed before blocking the
network threads</td>
+      <td>The number of requests that can be queued up for processing by
the I/O threads before the network threads stop reading in new
requests.</td>
     </tr>
     <tr>
-      <td>port</td>
-      <td>6667</td>
-      <td>The port to listen and accept connections on</td>
-    </tr>
-    <tr>
       <td>host.name</td>
       <td>null</td>
       <td>
-        <p>Hostname of broker. If this is set, it will only bind to this
address. If this is not set, it will bind to all interfaces, and publish
one to ZK</p>
+        <p>Hostname of broker. If this is set, it will only bind to this
address. If this is not set, it will bind to all interfaces, and publish
one to ZK.</p>
      </td>
     </tr>
     <tr>
       <td>socket.send.buffer.bytes</td>
       <td>100 * 1024</td>
-      <td>The SO_SNDBUFF buffer of the socket sever sockets</td>
+      <td>The SO_SNDBUFF buffer the server prefers for socket
connections.</td>
     </tr>
     <tr>
       <td>socket.receive.buffer.bytes</td>
       <td>100 * 1024</td>
-      <td>The SO_RCVBUFF buffer of the socket sever sockets</td>
+      <td>The SO_RCVBUFF buffer the server prefers for socket
connections.</td>
     </tr>
     <tr>
       <td>socket.request.max.bytes</td>
       <td>100 * 1024 * 1024</td>
-      <td>The maximum number of bytes in a socket request</td>
+      <td>The maximum request size the server will allow. This prevents
the server from running out of memory and should be smaller than the Java
heap size.</td>
     </tr>
     <tr>
       <td>num.partitions</td>
       <td>1</td>
-      <td>The default number of log partitions per topic</td>
+      <td>The default number of partitions per topic.</td>
     </tr>
     <tr>
       <td>log.segment.bytes</td>
       <td nowrap>1024 * 1024 * 1024</td>
-      <td>The maximum size of a single log file</td>
+      <td>The log for a topic partition is stored as a directory of
segment files. This setting controls the size to which a segment file will
grow before a new segment is rolled over in the log.</td>
     </tr>
     <tr>
       <td>log.segment.bytes.per.topic</td>
       <td>""</td>
-      <td>The maximum size of a single log file for some specific
topic</td>
+      <td>This setting allows overriding log.segment.bytes on a per-topic
basis</td>
     </tr>
     <tr>
       <td>log.roll.hours</td>
       <td>24 * 7</td>
-      <td>The maximum time before a new log segment is rolled out</td>
+      <td>This setting will force Kafka to roll a new log segment even if
the log.segment.bytes size has not been reached.</td>
     </tr>
     <tr>
       <td>log.roll.hours.per.topic</td>
       <td>""</td>
-      <td>The number of hours before rolling out a new log segment for
some specific topic</td>
+      <td>This setting allows overriding log.roll.hours on a per-topic
basis.</td>
     </tr>
     <tr>
       <td>log.retention.hours</td>
       <td>24 * 7</td>
-      <td>The number of hours to keep a log file before deleting it</td>
+      <td>The number of hours to keep a log segment before it is deleted,
i.e. the default data retention window for all topics. Note that if both
log.retention.hours and log.retention.bytes are both set we delete a
segment when either limit is exceeded.</td>
     </tr>
     <tr>
       <td>log.retention.hours.per.topic</td>
       <td>""</td>
-      <td>The number of hours to keep a log file before deleting it for
some specific topic</td>
+      <td>A per-topic override for log.retention.hours.</td>
     </tr>
     <tr>
       <td>log.retention.bytes</td>
       <td>-1</td>
-      <td>The maximum size of the log per partition</td>
+      <td>The amount of data to retain in the log for each
topic-partitions. Note that this is the limit per-partition so multiple by
the number of partitions to get the total data retained for the topic. Also
note that if both log.retention.hours and log.retention.bytes are both set
we delete a segment when either limit is exceeded.</td>
     </tr>
     <tr>
       <td>log.retention.bytes.per.topic</td>
       <td>""</td>
-      <td>The maximum size of the log for each partition in some specific
topics</td>
+      <td>A per-topic override for log.retention.bytes.</td>
     </tr>
     <tr>
       <td>log.cleanup.interval.mins</td>
       <td>10</td>
-      <td>The frequency in minutes that the log cleaner checks whether any
log is eligible for deletion</td>
+      <td>The frequency in minutes that the log cleaner checks whether any
log segment is eligible for deletion to meet the retention policies.</td>
     </tr>
     <tr>
       <td>log.index.size.max.bytes</td>
       <td>10 * 1024 * 1024</td>
-      <td>The maximum size in bytes of the offset index</td>
+      <td>The maximum size in bytes we allow for the offset index for each
log segment. Note that we will always pre-allocate a sparse file with this
much space and shrink it down when the log rolls. If the index fills up we
will roll a new log segment even if we haven't reached the
log.segment.bytes limit.</td>
     </tr>
     <tr>
       <td>log.index.interval.bytes</td>
       <td>4096</td>
-      <td>The interval with which we add an entry to the offset index</td>
+      <td>The byte interval at which we add an entry to the offset index.
When executing a fetch request the server must do a linear scan for up to
this many bytes to find the correct position in the log to begin and end
the fetch. So setting this value to be larger will mean larger index files
(and a bit more memory usage) but less scanning. However the server will
never add more than one index entry per log append (even if more than
log.index.interval worth of messages are appended). In general you probably
don't need to mess with this value.</td>
     </tr>
     <tr>
       <td>log.flush.interval.messages</td>
       <td>10000</td>
-      <td>The number of messages accumulated on a log partition before
messages are flushed to disk</td>
+      <td>The number of messages written to a log partition before we
force an fsync on the log. Setting this higher will improve performance a
lot but will increase the window of data at risk in the event of a crash
(though that is usually best addressed through replication). If both this
setting and log.flush.interval.ms are both used the log will be flushed
when either criteria is met.</td>
     </tr>
     <tr>
       <td>log.flush.interval.ms.per.topic</td>
       <td>""</td>
-      <td>The maximum time in ms that a message in selected topics is kept
in memory before flushed to disk, e.g., topic1:3000,topic2:6000</td>
+      <td>The per-topic override for log.flush.interval.messages, e.g.,
topic1:3000,topic2:6000</td>
     </tr>
     <tr>
       <td>log.flush.scheduler.interval.ms</td>
       <td>3000</td>
-      <td>The frequency in ms that the log flusher checks whether any log
needs to be flushed to disk</td>
+      <td>The frequency in ms that the log flusher checks whether any log
is eligible to be flushed to disk.</td>
     </tr>
     <tr>
       <td>log.flush.interval.ms</td>
       <td>3000
      </td>
-      <td>The maximum time in ms that a message in any topic is kept in
memory before flushed to disk</td>
+      <td>The maximum time between fsync calls on the log. If used in
conjuction with log.flush.interval.messages the log will be flushed when
either criteria is met.</td>
     </tr>
     <tr>
       <td>auto.create.topics.enable</td>
       <td>true</td>
-      <td>Enable auto creation of topic on the server</td>
+      <td>Enable auto creation of topic on the server.  If this is set to
true then attempts to produce, consume, or fetch metadata for a
non-existent topic will automatically create it with the default
replication factor and number of partitions.</td>
     </tr>
     <tr>
       <td>controller.socket.timeout.ms</td>
       <td>30000</td>
-      <td>The socket timeout for controller-to-broker channels</td>
+      <td>The socket timeout for commands from the partition management
controller to the replicas.</td>
     </tr>
     <tr>
       <td>controller.message.queue.size</td>
@@ -177,74 +180,74 @@
     <tr>
       <td>default.replication.factor</td>
       <td>1</td>
-      <td>Default replication factors for automatically created topics</td>
+      <td>The default replication factor for automatically created
topics.</td>
     </tr>
     <tr>
       <td>replica.lag.time.max.ms</td>
       <td>10000</td>
-      <td>If a follower hasn't sent any fetch requests during this time,
the leader will remove the follower from isr</td>
+      <td>If a follower hasn't sent any fetch requests for this window of
time, the leader will remove the follower from ISR and treat it as
dead.</td>
     </tr>
     <tr>
       <td>replica.lag.max.messages</td>
       <td>4000</td>
-      <td>If the lag in messages between a leader and a follower exceeds
this number, the leader will remove the follower from isr</td>
+      <td>If a replica falls more than this many messages behind the
leader, the leader will remove the follower from ISR and treat it as
dead.</td>
     </tr>
     <tr>
       <td>replica.socket.timeout.ms</td>
       <td>30 * 1000</td>
-      <td>The socket timeout for network requests</td>
+      <td>The socket timeout for network requests to the leader for
replicating data.</td>
     </tr>
     <tr>
       <td>replica.socket.receive.buffer.bytes</td>
       <td>64 * 1024</td>
-      <td>The socket receive buffer for network requests</td>
+      <td>The socket receive buffer for network requests to the leader for
replicating data.</td>
     </tr>
     <tr>
       <td>replica.fetch.max.bytes</td>
       <td nowrap>1024 * 1024</td>
-      <td>The number of byes of messages to attempt to fetch</td>
+      <td>The number of byes of messages to attempt to fetch for each
partition in the fetch requests the replicas send to the leader.</td>
     </tr>
     <tr>
       <td>replica.fetch.wait.max.ms</td>
       <td>500</td>
-      <td>Max wait time for each fetcher request issued by follower
replicas</td>
+      <td>The maximum amount of time to wait time for data to arrive on
the leader in the fetch requests sent by the replicas to the leader.</td>
     </tr>
     <tr>
       <td>replica.fetch.min.bytes</td>
       <td>1</td>
-      <td>Minimum bytes expected for each fetch response. If not enough
bytes, wait up to replicaMaxWaitTimeMs</td>
+      <td>Minimum bytes expected for each fetch response for the fetch
requests from the replica to the leader. If not enough bytes, wait up to
replica.fetch.wait.max.ms for this many bytes to arrive.</td>
     </tr>
     <tr>
       <td>num.replica.fetchers</td>
       <td>1</td>
       <td>
-        <p>Number of fetcher threads used to replicate messages from a
source broker. Increasing this value can increase the degree of I/O
parallelism in the follower broker.</p>
+        <p>Number of threads used to replicate messages from leaders.
Increasing this value can increase the degree of I/O parallelism in the
follower broker.</p>
      </td>
     </tr>
     <tr>
       <td>replica.high.watermark.checkpoint.interval.ms</td>
       <td>5000</td>
-      <td>The frequency with which the high watermark is saved out to
disk</td>
+      <td>The frequency with which each replica saves its high watermark
to disk to handle recovery.</td>
     </tr>
     <tr>
       <td>fetch.purgatory.purge.interval.requests</td>
       <td>10000</td>
-      <td>The purge interval (in number of requests) of the fetch request
purgatory</td>
+      <td>The purge interval (in number of requests) of the fetch request
purgatory.</td>
     </tr>
     <tr>
       <td>producer.purgatory.purge.interval.requests</td>
       <td>10000</td>
-      <td>The purge interval (in number of requests) of the producer
request purgatory</td>
+      <td>The purge interval (in number of requests) of the producer
request purgatory.</td>
     </tr>
     <tr>
       <td>zookeeper.session.timeout.ms</td>
       <td>6000</td>
-      <td>Zookeeper session timeout</td>
+      <td>Zookeeper session timeout. If the server fails to heartbeat to
zookeeper within this period of time it is considered dead. If you set this
too low the server may be falsely considered dead; if you set it too high
it may take too long to recognize a truly dead server.</td>
     </tr>
     <tr>
       <td>zookeeper.connection.timeout.ms</td>
       <td>6000</td>
-      <td>The max time that the client waits to establish a connection to
zookeeper</td>
+      <td>The max time that the client waits to establish a connection to
zookeeper.</td>
     </tr>
     <tr>
       <td>zookeeper.sync.time.ms</td>
@@ -259,16 +262,17 @@
     <tr>
       <td>controlled.shutdown.max.retries</td>
       <td>3</td>
-      <td>Number of retries to complete the controlled shutdown
successfully</td>
+      <td>Number of retries to complete the controlled shutdown
successfully before executing an unclean shutdown.</td>
     </tr>
     <tr>
       <td>controlled.shutdown.retry.backoff.ms</td>
       <td>5000</td>
-      <td>Backoff time between two retries</td>
+      <td>Backoff time between shutdown retries.</td>
     </tr>
 </tbody></table>

 <p>More details about broker configuration can be found in the scala class
<code>kafka.server.KafkaConfig</code>.</p>
+
 <h3><a id="consumerconfigs">3.2 Consumer Configs</a></h3>
 The essential consumer configurations are the following:
 <ul>
@@ -285,7 +289,7 @@
     <tr>
       <td>group.id</td>
       <td colspan="1"></td>
-      <td>A string that uniquely identifies a set of consumers within the
same consumer group</td>
+      <td>A string that uniquely identifies the group of consumer
processes to which this consumer belongs. By setting the same group id
multiple processes indicate that they are all part of the same consumer
group.</td>
     </tr>
     <tr>
       <td>zookeeper.connect</td>
@@ -314,17 +318,17 @@
     <tr>
       <td>fetch.message.max.bytes</td>
       <td nowrap>1024 * 1024</td>
-      <td>The number of byes of messages to attempt to fetch</td>
+      <td>The number of byes of messages to attempt to fetch for each
topic-partition in each fetch request. These bytes will be read into memory
for each partition, so this helps control the memory used by the consumer.
The fetch request size must be at least as large as the maximum message
size the server allows or else it is possible for the producer to send
messages larger than the consumer can fetch.</td>
     </tr>
     <tr>
       <td>auto.commit.enable</td>
       <td colspan="1">true</td>
-      <td>If true, periodically commit to zookeeper the offset of messages
already fetched by the consumer</td>
+      <td>If true, periodically commit to zookeeper the offset of messages
already fetched by the consumer. This committed offset will be used when
the process fails as the position from which the new consumer will
begin.</td>
     </tr>
     <tr>
       <td>auto.commit.interval.ms</td>
       <td colspan="1">60 * 1000</td>
-      <td>The frequency in ms that the consumer offsets are committed to
zookeeper</td>
+      <td>The frequency in ms that the consumer offsets are committed to
zookeeper.</td>
     </tr>
     <tr>
       <td>queued.max.message.chunks</td>
@@ -334,12 +338,12 @@
     <tr>
       <td>rebalance.max.retries</td>
       <td colspan="1">4</td>
-      <td>Max number of retries during rebalance</td>
+      <td>When a new consumer joins a consumer group the set of consumers
attempt to "rebalance" the load to assign partitions to each consumer. If
the set of consumers changes while this assignment is taking place the
rebalance will fail and retry. This setting controls the maximum number of
attempts before giving up.</td>
     </tr>
     <tr>
       <td>fetch.min.bytes</td>
       <td colspan="1">1</td>
-      <td>The minimum amount of data the server should return for a fetch
request. If insufficient data is available the request will block</td>
+      <td>The minimum amount of data the server should return for a fetch
request. If insufficient data is available the request will wait for that
much data to accumulate before answering the request.</td>
     </tr>
     <tr>
       <td>fetch.wait.max.ms</td>
@@ -349,12 +353,12 @@
     <tr>
       <td>rebalance.backoff.ms</td>
       <td>2000</td>
-      <td>Backoff time between retries during rebalance</td>
+      <td>Backoff time between retries during rebalance.</td>
     </tr>
     <tr>
       <td>refresh.leader.backoff.ms</td>
       <td colspan="1">200</td>
-      <td>Backoff time to refresh the leader of a partition after it loses
the current leader</td>
+      <td>Backoff time to wait before trying to determine the leader of a
partition that has just lost its leader.</td>
     </tr>
     <tr>
       <td>auto.offset.reset</td>
@@ -370,18 +374,18 @@
     </tr>
     <tr>
       <td>client.id</td>
-      <td colspan="1">${group.id}</td>
-      <td>Client id is specified by the kafka consumer client, used to
distinguish different clients</td>
+      <td colspan="1">group id value</td>
+      <td>The client id is a user-specified string sent in each request to
help trace calls. It should logically identify the application making the
request.</td>
     </tr>
     <tr>
       <td>zookeeper.session.timeout.ms </td>
       <td colspan="1">6000</td>
-      <td>Zookeeper session timeout</td>
+      <td>Zookeeper session timeout. If the consumer fails to heartbeat to
zookeeper for this period of time it is considered dead and a rebalance
will occur.</td>
     </tr>
     <tr>
       <td>zookeeper.connection.timeout.ms</td>
       <td colspan="1">6000</td>
-      <td>The max time that the client waits to establish a connection to
zookeeper</td>
+      <td>The max time that the client waits while establishing a
connection to zookeeper.</td>
     </tr>
     <tr>
       <td>zookeeper.sync.time.ms </td>
@@ -419,35 +423,46 @@
       <td>request.required.acks</td>
       <td colspan="1">0</td>
       <td>
-        <p>This value controls when the producer receives an
acknowledgement from the broker. Typical values are (1) 0, which means that
the producer never waits for an acknowledgement from the broker (the same
behavior as 0.7); (2) 1, which means that the producer gets an
acknowledgement after the leader replica has received the data; (3) -1,
which means that the producer gets an acknowledgement after all in-sync
replicas have received the data. The first option provides the lowest
latency (no network delay), but the worst durability (some data loss when
the leader replica fails). The second option provides lower latency (one
network round trip) and better durability (few data loss when the leader
replica fails). The last option provides low latency (two network round
trips) and the best durability (no data loss as long as the number of
failed brokers is less the replication factor of the topic).</p>
+        <p>This value controls when a produce request is considered
completed. Specifically, how many other brokers must have committed the
data to their log and acknowledged this to the leader? Typical values are
+       <ul>
+     <li>0, which means that the producer never waits for an
acknowledgement from the broker (the same behavior as 0.7). This option
provides the lowest latency but the weakest durability guarantees (some
data will be lost when a server fails).
+ <li> 1, which means that the producer gets an acknowledgement after the
leader replica has received the data. This option provides better
durability as the client waits until the server acknowledges the request as
successful (only messages that were written to the now-dead leader but not
yet replicated will be lost).
+ <li> -1, which means that the producer gets an acknowledgement after all
in-sync replicas have received the data. This option provides the best
durability, we guarantee that no messages will be lost as long as at least
one in sync replica remains.
+ </ul>
+ </p>
      </td>
     </tr>
     <tr>
+      <td>request.timeout.ms</td>
+      <td colspan="1">1500</td>
+      <td>The amount of time the broker will wait trying to meet the
request.required.acks requirement before sending back an error to the
client.</td>
+    </tr>
+    <tr>
       <td>producer.type</td>
       <td colspan="1">sync</td>
       <td>
-        <p>This parameter specifies whether the messages are sent
asynchronously or not. Valid values are (1) async for asynchronous send and
(2) sync for synchronous send.</p>
+        <p>This parameter specifies whether the messages are sent
asynchronously in a background thread. Valid values are (1) async for
asynchronous send and (2) sync for synchronous send. By setting the
producer to async we allow batching together of requests (which is great
for throughput) but open the possibility of a failure of the client machine
dropping unsent data.</p>
      </td>
     <tr>
       <td>serializer.class</td>
-      <td colspan="1">DefaultEncoder</td>
+      <td colspan="1">kafka.serializer.DefaultEncoder</td>
       <td>The serializer class for messages. The default encoder takes a
byte[] and returns the same byte[].</td>
     </tr>
     <tr>
       <td>key.serializer.class</td>
-      <td colspan="1">${serializer.class}</td>
-      <td>The serializer class for keys (defaults to the same as for
messages)</td>
+      <td colspan="1"></td>
+      <td>The serializer class for keys (defaults to the same as for
messages if nothing is given).</td>
     </tr>
     <tr>
       <td>partitioner.class</td>
-      <td colspan="1">DefaultPartitioner</td>
+      <td colspan="1">kafka.producer.DefaultPartitioner</td>
       <td>The partitioner class for partitioning messages amongst
sub-topics. The default partitioner is based on the hash of the key.</td>
     </tr>
     <tr>
       <td>compression.codec</td>
       <td colspan="1">none</td>
       <td>
-        <p>This parameter allows you to specify the compression codec for
all data generated by this producer. Valid values are none, gzip and
snappy.</p>
+        <p>This parameter allows you to specify the compression codec for
all data generated by this producer. Valid values are "none", "gzip" and
"snappy".</p>
      </td>
     </tr>
     <tr>
@@ -461,14 +476,14 @@
       <td>message.send.max.retries</td>
       <td colspan="1">3</td>
       <td>
-        <p>The leader may be unavailable transiently, which can fail the
sending of a message. This property specifies the number of retries when
such failures occur.</p>
+        <p>This property will cause the producer to automatically retry a
failed send request. This property specifies the number of retries when
such failures occur. Note that setting a non-zero value here can lead to
duplicates in the case of network errors that cause a message to be sent
but the acknowledgement to be lost.</p>
      </td>
     </tr>
     <tr>
       <td>retry.backoff.ms</td>
       <td colspan="1">100</td>
       <td>
-        <p>Before each retry, the producer refreshes the metadata of
relevant topics. Since leader election takes a bit of time, this property
specifies the amount of time that the producer waits before refreshing the
metadata.</p>
+        <p>Before each retry, the producer refreshes the metadata of
relevant topics to see if a new leader has been elected. Since leader
election takes a bit of time, this property specifies the amount of time
that the producer waits before refreshing the metadata.</p>
      </td>
     </tr>
     <tr>
@@ -481,24 +496,24 @@
     <tr>
       <td>queue.buffering.max.ms</td>
       <td colspan="1">5000</td>
-      <td>Maximum time, in milliseconds, for buffering data on the
producer queue</td>
+      <td>Maximum time to buffer data when using async mode. For example a
setting of 100 will try to batch together 100ms of messages to send at
once. This will improve throughput but adds message delivery latency due to
the buffering.</td>
     </tr>
     <tr>
       <td>queue.buffering.max.messages</td>
       <td colspan="1">10000</td>
-      <td>The maximum size of the blocking queue for buffering on the
producer</td>
+      <td>The maximum number of unsent messages that can be queued up the
producer when using async mode before either the producer must be blocked
or data must be dropped.</td>
     </tr>
     <tr>
       <td>queue.enqueue.timeout.ms</td>
       <td colspan="1">-1</td>
       <td>
-        <p>Timeout for event enqueue:<br/> * 0: events will be enqueued
immediately or dropped if the queue is full<br/> * -ve: enqueue will block
indefinitely if the queue is full<br/> * +ve: enqueue will block up to this
many milliseconds if the queue is full</p>
+        <p>The amount of time to block before dropping messages when
running in async mode and the buffer has reached
queue.buffering.max.messages. If set to 0 events will be enqueued
immediately or dropped if the queue is full (the producer send call will
never block). If set to -1 the producer will block indefinitely and never
willingly drop a send.</p>
      </td>
     </tr>
     <tr>
       <td>batch.num.messages</td>
       <td colspan="1">200</td>
-      <td>The number of messages batched at the producer</td>
+      <td>The number of messages to send in one batch when using async
mode. The producer will wait until either this number of messages are ready
to send or queue.buffer.max.ms is reached.</td>
     </tr>
     <tr>
       <td>send.buffer.bytes</td>
@@ -508,12 +523,7 @@
     <tr>
       <td>client.id</td>
       <td colspan="1">""</td>
-      <td>The client application sending the producer requests</td>
+      <td>The client id is a user-specified string sent in each request to
help trace calls. It should logically identify the application making the
request.</td>
     </tr>
-    <tr>
-      <td>request.timeout.ms</td>
-      <td colspan="1">1500</td>
-      <td>The ack timeout of the producer requests. Value must be
non-negative and non-zero</td>
-    </tr>
 </tbody></table>
 <p>More details about producer configuration can be found in the scala
class <code>kafka.producer.ProducerConfig</code>.</p>

Mime
  • Unnamed multipart/alternative (inline, None, 0 bytes)
View raw message