kafka-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From gwens...@apache.org
Subject [50/51] [partial] kafka-site git commit: KAFKA-2425: Initial upload of Kafka documentation to git repository with intent to replace SVN
Date Fri, 02 Oct 2015 19:19:48 GMT
http://git-wip-us.apache.org/repos/asf/kafka-site/blob/ba6c994c/08/configuration.html
----------------------------------------------------------------------
diff --git a/08/configuration.html b/08/configuration.html
new file mode 100644
index 0000000..6264f72
--- /dev/null
+++ b/08/configuration.html
@@ -0,0 +1,529 @@
+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>
+	<li><code>broker.id</code>
+	<li><code>log.dirs</code>
+	<li><code>zookeeper.connect</code>
+</ul>
+
+<table class="data-table">
+<tbody><tr>
+      <th>Property</th>
+      <th>Default</th>
+      <th>Description</th>
+    </tr>
+    <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. 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>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>.
+	<p>
+Zookeeper also allows you to add a "chroot" path which will make all kafka data for this cluster appear under a particular path. This is a way to setup multiple Kafka clusters or other applications on the same zookeeper cluster. To do this give a connection string in the form <code>hostname1:port1,hostname2:port2,hostname3:port3/chroot/path</code> which would put all this cluster's data under the path <code>/chroot/path</code>. Note that you must create this path yourself prior to starting the broker and consumers must use the same connection string.</td>
+    </tr>
+    <tr>
+      <td>message.max.bytes</td>
+      <td>1000000</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 producer 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. You probably don't need to change this.</td>
+    </tr>
+    <tr>
+      <td>num.io.threads</td>
+      <td>8</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 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>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>
+     </td>
+    </tr>
+    <tr>
+      <td>socket.send.buffer.bytes</td>
+      <td>100 * 1024</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 the server prefers for socket connections.</td>
+    </tr>
+    <tr>
+      <td>socket.request.max.bytes</td>
+      <td>100 * 1024 * 1024</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 partitions per topic.</td>
+    </tr>
+    <tr>
+      <td>log.segment.bytes</td>
+      <td nowrap>1024 * 1024 * 1024</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>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>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>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 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>A per-topic override for log.retention.hours.</td>
+    </tr>
+    <tr>
+      <td>log.retention.bytes</td>
+      <td>-1</td>
+      <td>The amount of data to retain in the log for each topic-partitions. Note that this is the limit per-partition so multiply 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>A per-topic override for log.retention.bytes.</td>
+    </tr>
+    <tr>
+      <td>log.retention.check.interval.ms</td>
+      <td>300000</td>
+      <td>The frequency in milliseconds 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 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 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 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 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 is eligible to be flushed to disk.</td>
+    </tr>
+    <tr>
+      <td>log.flush.interval.ms</td>
+      <td>3000
+     </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.  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 commands from the partition management controller to the replicas.</td>
+    </tr>
+    <tr>
+      <td>controller.message.queue.size</td>
+      <td>10</td>
+      <td>The buffer size for controller-to-broker-channels</td>
+    </tr>
+    <tr>
+      <td>default.replication.factor</td>
+      <td>1</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 for this window of time, the leader will remove the follower from ISR (in-sync replicas) and treat it as dead.</td>
+    </tr>
+    <tr>
+      <td>replica.lag.max.messages</td>
+      <td>4000</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 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 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 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>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 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 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 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>
+    </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>
+    </tr>
+    <tr>
+      <td>zookeeper.session.timeout.ms</td>
+      <td>6000</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 maximum amount of time that the client waits to establish a connection to zookeeper.</td>
+    </tr>
+    <tr>
+      <td>zookeeper.sync.time.ms</td>
+      <td>2000</td>
+      <td>How far a ZK follower can be behind a ZK leader.</td>
+    </tr>
+    <tr>
+      <td>controlled.shutdown.enable</td>
+      <td>false</td>
+      <td>Enable controlled shutdown of the broker. If enabled, the broker will move all leaders on it to some other brokers before shutting itself down. This reduces the unavailability window during shutdown.</td>
+    </tr>
+    <tr>
+      <td>controlled.shutdown.max.retries</td>
+      <td>3</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 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>
+        <li><code>group.id</code>
+        <li><code>zookeeper.connect</code>
+</ul>
+
+<table class="data-table">
+<tbody><tr>
+        <th>Property</th>
+        <th>Default</th>
+        <th>Description</th>
+</tr>
+    <tr>
+      <td>group.id</td>
+      <td colspan="1"></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>
+      <td colspan="1"></td>
+	      <td>Specifies the zookeeper connection string in the form <code>hostname:port</code> where host and port are the host and port of a zookeeper server. To allow connecting through other zookeeper nodes when that zookeeper machine is down you can also specify multiple hosts in the form <code>hostname1:port1,hostname2:port2,hostname3:port3</code>.
+		<p>
+	The server may also have a zookeeper chroot path as part of it's zookeeper connection string which puts its data under some path in the global zookeeper namespace. If so the consumer should use the same chroot path in its connection string. For example to give a chroot path of <code>/chroot/path</code> you would give the connection string as  <code>hostname1:port1,hostname2:port2,hostname3:port3/chroot/path</code>.</td>
+    </tr>
+    <tr>
+      <td>consumer.id</td>
+      <td colspan="1">null</td>
+      <td>
+        <p>Generated automatically if not set.</p>
+     </td>
+    </tr>
+    <tr>
+      <td>socket.timeout.ms</td>
+      <td colspan="1">30 * 1000</td>
+      <td>The socket timeout for network requests. The actual timeout set will be max.fetch.wait + socket.timeout.ms.</td>
+    </tr>
+    <tr>
+      <td>socket.receive.buffer.bytes</td>
+      <td colspan="1">64 * 1024</td>
+      <td>The socket receive buffer for network requests</td>
+    </tr>
+    <tr>
+      <td>fetch.message.max.bytes</td>
+      <td nowrap>1024 * 1024</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. 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>
+    </tr>
+    <tr>
+      <td>queued.max.message.chunks</td>
+      <td colspan="1">10</td>
+      <td>Max number of message chunks buffered for consumption. Each chunk can be up to fetch.message.max.bytes.</td>
+    </tr>
+    <tr>
+      <td>rebalance.max.retries</td>
+      <td colspan="1">4</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 wait for that much data to accumulate before answering the request.</td>
+    </tr>
+    <tr>
+      <td>fetch.wait.max.ms</td>
+      <td colspan="1">100</td>
+      <td>The maximum amount of time the server will block before answering the fetch request if there isn't sufficient data to immediately satisfy fetch.min.bytes</td>
+    </tr>
+    <tr>
+      <td>rebalance.backoff.ms</td>
+      <td>2000</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 wait before trying to determine the leader of a partition that has just lost its leader.</td>
+    </tr>
+    <tr>
+      <td>auto.offset.reset</td>
+      <td colspan="1">largest</td>
+      <td>
+        <p>What to do when there is no initial offset in Zookeeper or if an offset is out of range:<br/>* smallest : automatically reset the offset to the smallest offset<br/>* largest : automatically reset the offset to the largest offset<br/>* anything else: throw exception to the consumer. If this is set to largest, the consumer may lose some messages when the number of partitions, for the topics it subscribes to, changes on the broker. To prevent data loss during partition addition, set auto.offset.reset to smallest</p>
+     </td>
+    </tr>
+    <tr>
+      <td>consumer.timeout.ms</td>
+      <td colspan="1">-1</td>
+      <td>Throw a timeout exception to the consumer if no message is available for consumption after the specified interval</td>
+    </tr>
+    <tr>
+      <td>client.id</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. 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 while establishing a connection to zookeeper.</td>
+    </tr>
+    <tr>
+      <td>zookeeper.sync.time.ms </td>
+      <td colspan="1">2000</td>
+      <td>How far a ZK follower can be behind a ZK leader</td>
+    </tr>
+</tbody>
+</table>
+
+
+<p>More details about consumer configuration can be found in the scala class <code>kafka.consumer.ConsumerConfig</code>.</p>
+<h3><a id="producerconfigs">3.3 Producer Configs</a></h3>
+Essential configuration properties for the producer include:
+<ul>
+        <li><code>metadata.broker.list</code>
+        <li><code>request.required.acks</code>
+        <li><code>producer.type</code>
+        <li><code>serializer.class</code>
+</ul>
+
+<table class="data-table">
+<tbody><tr>
+        <th>Property</th>
+        <th>Default</th>
+        <th>Description</th>
+      </tr>
+    <tr>
+      <td>metadata.broker.list</td>
+      <td colspan="1"></td>
+      <td>
+        <p>This is for bootstrapping and the producer will only use it for getting metadata (topics, partitions and replicas). The socket connections for sending the actual data will be established based on the broker information returned in the metadata. The format is host1:port1,host2:port2, and the list can be a subset of brokers or a VIP pointing to a subset of brokers.</p>
+     </td>
+    </tr>
+    <tr>
+      <td>request.required.acks</td>
+      <td colspan="1">0</td>
+      <td>
+        <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">10000</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 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">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"></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">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>
+     </td>
+    </tr>
+    <tr>
+      <td>compressed.topics</td>
+      <td colspan="1">null</td>
+      <td>
+        <p>This parameter allows you to set whether compression should be turned on for particular topics. If the compression codec is anything other than NoCompressionCodec, enable compression only for specified topics if any. If the list of compressed topics is empty, then enable the specified compression codec for all topics. If the compression codec is NoCompressionCodec, compression is disabled for all topics</p>
+     </td>
+    </tr>
+    <tr>
+      <td>message.send.max.retries</td>
+      <td colspan="1">3</td>
+      <td>
+        <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 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>
+      <td>topic.metadata.refresh.interval.ms</td>
+      <td colspan="1">600 * 1000</td>
+      <td>
+        <p>The producer generally refreshes the topic metadata from brokers when there is a failure (partition missing, leader not available...). It will also poll regularly (default: every 10min so 600000ms). If you set this to a negative value, metadata will only get refreshed on failure. If you set this to zero, the metadata will get refreshed after each message sent (not recommended). Important note: the refresh happen only AFTER the message is sent, so if the producer never sends a message the metadata is never refreshed</p>
+     </td>
+    </tr>
+    <tr>
+      <td>queue.buffering.max.ms</td>
+      <td colspan="1">5000</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 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>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 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>
+      <td colspan="1">100 * 1024</td>
+      <td>Socket write buffer size</td>
+    </tr>
+    <tr>
+      <td>client.id</td>
+      <td colspan="1">""</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>
+</tbody></table>
+<p>More details about producer configuration can be found in the scala class <code>kafka.producer.ProducerConfig</code>.</p>

http://git-wip-us.apache.org/repos/asf/kafka-site/blob/ba6c994c/08/design.html
----------------------------------------------------------------------
diff --git a/08/design.html b/08/design.html
new file mode 100644
index 0000000..b1d2355
--- /dev/null
+++ b/08/design.html
@@ -0,0 +1,232 @@
+<h3><a id="majordesignelements">4.1 Motivation</a></h3>
+<p>
+We designed Kafka to be able to act as a unified platform for handling all the real-time data feeds <a href="#introduction">a large company might have</a>. To do this we had to think through a fairly broad set of use cases.
+<p>
+It would have to have high-throughput to support high volume event streams such as real-time log aggregation.
+<p>
+It would need to deal gracefully with large data backlogs to be able to support periodic data loads from offline systems.
+<p>
+It also meant the system would have to handle low-latency delivery to handle more traditional messaging use-cases.
+<p>
+We wanted to support partitioned, distributed, real-time processing of these feeds to create new, derived feeds. This motivated our partitioning and consumer model.
+<p>
+Finally in cases where the stream is fed into other data systems for serving we knew the system would have to be able to guarantee fault-tolerance in the presence of machine failures.
+<p>
+Supporting these uses led us to a design with a number of unique elements, more akin to a database log then a traditional messaging system. We will outline some elements of the design in the following sections.
+
+<h3><a id="persistence">4.2 Persistence</a></h3>
+<h4>Don't fear the filesystem!</h4>
+<p>
+Kafka relies heavily on the filesystem for storing and caching messages. There is a general perception that "disks are slow" which makes people skeptical that a persistent structure can offer competitive performance. In fact disks are both much slower and much faster than people expect depending on how they are used; and a properly designed disk structure can often be as fast as the network.
+<p>
+The key fact about disk performance is that the throughput of hard drives has been diverging from the latency of a disk seek for the last decade. As a result the performance of linear writes on a <a href="http://en.wikipedia.org/wiki/Non-RAID_drive_architectures">JBOD</a> configuration with six 7200rpm SATA RAID-5 array is about 600MB/sec but the performance of random writes is only about 100k/sec&mdash;a difference of over 6000X. These linear reads and writes are the most predictable of all usage patterns, and are heavily optimized by the operating system. A modern operating system provides read-ahead and write-behind techniques that prefetch data in large block multiples and group smaller logical writes into large physical writes. A further discussion of this issue can be found in this <a href="http://queue.acm.org/detail.cfm?id=1563874">ACM Queue article</a>; they actually find that <a href="http://deliveryimages.acm.org/10.1145/1570000/1563874/jacobs3.jpg">sequential disk access
  can in some cases be faster than random memory access!</a>
+<p>
+To compensate for this performance divergence modern operating systems have become increasingly aggressive in their use of main memory for disk caching. A modern OS will happily divert <i>all</i> free memory to disk caching with little performance penalty when the memory is reclaimed. All disk reads and writes will go through this unified cache. This feature cannot easily be turned off without using direct I/O, so even if a process maintains an in-process cache of the data, this data will likely be duplicated in OS pagecache, effectively storing everything twice.
+<p>
+Furthermore we are building on top of the JVM, and anyone who has spent any time with Java memory usage knows two things:
+<ol>
+	<li>The memory overhead of objects is very high, often doubling the size of the data stored (or worse).</li>
+	<li>Java garbage collection becomes increasingly fiddly and slow as the in-heap data increases.</li>
+</ol>
+<p>
+As a result of these factors using the filesystem and relying on pagecache is superior to maintaining an in-memory cache or other structure&mdash;we at least double the available cache by having automatic access to all free memory, and likely double again by storing a compact byte structure rather than individual objects. Doing so will result in a cache of up to 28-30GB on a 32GB machine without GC penalties. Furthermore this cache will stay warm even if the service is restarted, whereas the in-process cache will need to be rebuilt in memory (which for a 10GB cache may take 10 minutes) or else it will need to start with a completely cold cache (which likely means terrible initial performance). This also greatly simplifies the code as all logic for maintaining coherency between the cache and filesystem is now in the OS, which tends to do so more efficiently and more correctly than one-off in-process attempts. If your disk usage favors linear reads then read-ahead is effectively pre-p
 opulating this cache with useful data on each disk read.
+<p>
+This suggests a design which is very simple: rather than maintain as much as possible in-memory and flush it all out to the filesystem in a panic when we run out of space, we invert that. All data is immediately written to a persistent log on the filesystem without necessarily flushing to disk. In effect this just means that it is transferred into the kernel's pagecache.
+<p>
+This style of pagecache-centric design is described in an <a href="http://varnish.projects.linpro.no/wiki/ArchitectNotes">article</a> on the design of Varnish here (along with a healthy dose of arrogance).
+
+<h4>Constant Time Suffices</h4>
+<p>
+The persistent data structure used in messaging systems are often a per-consumer queue with an associated BTree or other general-purpose random access data structures to maintain metadata about messages. BTrees are the most versatile data structure available, and make it possible to support a wide variety of transactional and non-transactional semantics in the messaging system. They do come with a fairly high cost, though: Btree operations are O(log N). Normally O(log N) is considered essentially equivalent to constant time, but this is not true for disk operations. Disk seeks come at 10 ms a pop, and each disk can do only one seek at a time so parallelism is limited. Hence even a handful of disk seeks leads to very high overhead. Since storage systems mix very fast cached operations with very slow physical disk operations, the observed performance of tree structures is often superlinear as data increases with fixed cache--i.e. doubling your data makes things much worse then twice a
 s slow.
+<p>
+Intuitively a persistent queue could be built on simple reads and appends to files as is commonly the case with logging solutions. This structure has the advantage that all operations are O(1) and reads do not block writes or each other. This has obvious performance advantages since the performance is completely decoupled from the data size&mdash;one server can now take full advantage of a number of cheap, low-rotational speed 1+TB SATA drives. Though they have poor seek performance, these drives have acceptable performance for large reads and writes and come at 1/3 the price and 3x the capacity.
+<p>
+Having access to virtually unlimited disk space without any performance penalty means that we can provide some features not usually found in a messaging system. For example, in Kafka, instead of attempting to deleting messages as soon as they are consumed, we can retain messages for a relative long period (say a week). This leads to a great deal of flexibility for consumers, as we will describe.
+
+<h3><a id="maximizingefficiency">4.3 Efficiency</a></h3>
+<p>
+We have put significant effort into efficiency. One of our primary use cases is handling web activity data, which is very high volume: each page view may generate dozens of writes. Furthermore we assume each message published is read by at least one consumer (often many), hence we strive to make consumption as cheap as possible.
+<p>
+We have also found, from experience building and running a number of similar systems, that efficiency is a key to effective multi-tenant operations. If the downstream infrastructure service can easily become a bottleneck due to a small bump in usage by the application, such small changes will often create problems. By being very fast we help ensure that the application will tip-over under load before the infrastructure. This is particularly important when trying to run a centralized service that supports dozens or hundreds of applications on a centralized cluster as changes in usage patterns are a near-daily occurrence.
+<p>
+We discussed disk efficiency in the previous section. Once poor disk access patterns have been eliminated, there are two common causes of inefficiency in this type of system: too many small I/O operations, and excessive byte copying.
+<p>
+The small I/O problem happens both between the client and the server and in the server's own persistent operations.
+<p>
+To avoid this, our protocol is built around a "message set" abstraction that naturally groups messages together. This allows network requests to group messages together and amortize the overhead of the network roundtrip rather than sending a single message at a time. The server in turn appends chunks of messages to its log in one go, and the consumer fetches large linear chunks at a time.
+<p>
+This simple optimization produces orders of magnitude speed up. Batching leads to larger network packets, larger sequential disk operations, contiguous memory blocks, and so on, all of which allows Kafka to turn a bursty stream of random message writes into linear writes that flow to the consumers.
+<p>
+The other inefficiency is in byte copying. At low message rates this is not an issue, but under load the impact is significant. To avoid this we employ a standardized binary message format that is shared by the producer, the broker, and the consumer (so data chunks can be transferred without modification between them). 
+<p>
+The message log maintained by the broker is itself just a directory of files, each populated by a sequence of message sets that have been written to disk in the same format used by the producer and consumer. Maintaining this common format allows optimization of the most important operation: network transfer of persistent log chunks. Modern unix operating systems offer a highly optimized code path for transferring data out of pagecache to a socket; in Linux this is done with the <a href="http://man7.org/linux/man-pages/man2/sendfile.2.html">sendfile system call</a>.
+<p>
+To understand the impact of sendfile, it is important to understand the common data path for transfer of data from file to socket:
+<ol>
+	<li>The operating system reads data from the disk into pagecache in kernel space</li>
+	<li>The application reads the data from kernel space into a user-space buffer</li>
+	<li>The application writes the data back into kernel space into a socket buffer</li>
+	<li>The operating system copies the data from the socket buffer to the NIC buffer where it is sent over the network</li>
+</ol>
+<p>
+This is clearly inefficient, there are four copies and two system calls. Using sendfile, this re-copying is avoided by allowing the OS to send the data from pagecache to the network directly. So in this optimized path, only the final copy to the NIC buffer is needed.
+<p>
+We expect a common use case to be multiple consumers on a topic. Using the zero-copy optimization above, data is copied into pagecache exactly once and reused on each consumption instead of being stored in memory and copied out to kernel space every time it is read. This allows messages to be consumed at a rate that approaches the limit of the network connection.
+<p>
+This combination of pagecache and sendfile means that on a Kafka cluster where the consumers are mostly caught up you will see no read activity on the disks whatsoever as they will be serving data entirely from cache.
+<p>
+For more background on the sendfile and zero-copy support in Java, see this <a href="http://www.ibm.com/developerworks/linux/library/j-zerocopy">article</a>.
+
+<h4>End-to-end Batch Compression</h4>
+<p>
+In some cases the bottleneck is actually not CPU or disk but network bandwidth. This is particularly true for a data pipeline that needs to send messages between data centers over a wide-area network. Of course the user can always compress its messages one at a time without any support needed from Kafka, but this can lead to very poor compression ratios as much of the redundancy is due to repetition between messages of the same type (e.g. field names in JSON or user agents in web logs or common string values). Efficient compression requires compressing multiple messages together rather than compressing each message individually.
+<p>
+Kafka supports this by allowing recursive message sets. A batch of messages can be clumped together compressed and sent to the server in this form. This batch of messages will be written in compressed form and will remain compressed in the log and will only be decompressed by the consumer.
+<p>
+Kafka supports GZIP and Snappy compression protocols. More details on compression can be found <a href="https://cwiki.apache.org/confluence/display/KAFKA/Compression">here</a>.
+
+<h3><a id="theproducer">4.4 The Producer</a></h3>
+
+<h4>Load balancing</h4>
+<p>
+The producer sends data directly to the broker that is the leader for the partition without any intervening routing tier. To help the producer do this all Kafka nodes can answer a request for metadata about which servers are alive and where the leaders for the partitions of a topic are at any given time to allow the producer to appropriate direct its requests.
+<p>
+The client controls which partition it publishes messages to. This can be done at random, implementing a kind of random load balancing, or it can be done by some semantic partitioning function. We expose the interface for semantic partitioning by allowing the user to specify a key to partition by and using this to hash to a partition (there is also an option to override the partition function if need be). For example if the key chosen was a user id then all data for a given user would be sent to the same partition. This in turn will allow consumers to make locality assumptions about their consumption. This style of partitioning is explicitly designed to allow locality-sensitive processing in consumers.
+
+<h4>Asynchronous send</h4>
+<p>
+Batching is one of the big drivers of efficiency, and to enable batching the Kafka producer has an asynchronous mode that accumulates data in memory and sends out larger batches in a single request. The batching can be configured to accumulate no more than a fixed number of messages and to wait no longer than some fixed latency bound (say 100 messages or 5 seconds). This allows the accumulation of more bytes to send, and few larger I/O operations on the servers. Since this buffering happens in the client it obviously reduces the durability as any data buffered in memory and not yet sent will be lost in the event of a producer crash.
+
+<h3><a id="theconsumer">4.5 The Consumer</a></h3>
+
+The Kafka consumer works by issuing "fetch" requests to the brokers leading the partitions it wants to consume. The consumer specifies its position in the log with each request and receives back a chunk of log beginning at that position. The consumer thus has significant control over this position and can rewind it to re-consume data if need be.
+
+<h4>Push vs. pull</h4>
+<p>
+An initial question we considered is whether consumers should pull data from brokers or brokers should push data to the consumer. In this respect Kafka follows a more traditional design, shared by most messaging systems, where data is pushed to the broker from the producer and pulled from the broker by the consumer. Some logging-centric systems, such as <a href="http://github.com/facebook/scribe">scribe</a> and <a href="http://github.com/cloudera/flume">flume</a> follow a very different push based path where  data is pushed downstream. There are pros and cons to both approaches. However a push-based system has difficulty dealing with diverse consumers as the broker controls the rate at which data is transferred. The goal is generally for the consumer to be able to consume at the maximum possible rate; unfortunately in a push system this means the consumer tends to be overwhelmed when its rate of consumption falls below the rate of production (a denial of service attack, in essence).
  A pull-based system has the nicer property that the consumer simply falls behind and catches up when it can. This can be mitigated with some kind of backoff protocol by which the consumer can indicate it is overwhelmed, but getting the rate of transfer to fully utilize (but never over-utilize) the consumer is trickier than it seems. Previous attempts at building systems in this fashion led us to go with a more traditional pull model.
+<p>
+Another advantage of a pull-based system is that it lends itself to aggressive batching of data sent to the consumer. A push-based system must choose to either send a request immediately or accumulate more data and then send it later without knowledge of whether the downstream consumer will be able to immediately process it. If tuned for low latency this will result in sending a single message at a time only for the transfer to end up being buffered anyway, which is wasteful. A pull-based design fixes this as the consumer always pulls all available messages after its current position in the log (or up to some configurable max size). So one gets optimal batching without introducing unnecessary latency.
+<p>
+The deficiency of a naive pull-based system is that if the broker has no data the consumer may end up polling in a tight loop, effectively busy-waiting for data to arrive. To avoid this we have parameters in our pull request that allow the consumer request to block in a "long poll" waiting until data arrives (and optionally waiting until a given number of bytes is available to ensure large transfer sizes).
+<p>
+You could imagine other possible designs which would be only pull, end-to-end. The producer would locally write to a local log, and brokers would pull from that with consumers pulling from them. A similar type of "store-and-forward" producer is often proposed. This is intriguing but we felt not very suitable for our target use cases which have thousands of producers. Our experience running persistent data systems at scale led us to feel that involving thousands of disks in the system across many applications would not actually make things more reliable and would be a nightmare to operate. And in practice we have found that we can run a pipeline with strong SLAs at large scale without a need for producer persistence.
+
+<h4>Consumer Position</h4>
+Keeping track of <i>what</i> has been consumed, is, surprisingly, one of the key performance points of a messaging system.
+<p>
+Most messaging systems keep metadata about what messages have been consumed on the broker. That is, as a message is handed out to a consumer, the broker either records that fact locally immediately or it may wait for acknowledgement from the consumer. This is a fairly intuitive choice, and indeed for a single machine server it is not clear where else this state could go. Since the data structure used for storage in many messaging systems scale poorly, this is also a pragmatic choice--since the broker knows what is consumed it can immediately delete it, keeping the data size small.
+<p>
+What is perhaps not obvious, is that getting the broker and consumer to come into agreement about what has been consumed is not a trivial problem. If the broker records a message as <b>consumed</b> immediately every time it is handed out over the network, then if the consumer fails to process the message (say because it crashes or the request times out or whatever) that message will be lost. To solve this problem, many messaging systems add an acknowledgement feature which means that messages are only marked as <b>sent</b> not <b>consumed</b> when they are sent; the broker waits for a specific acknowledgement from the consumer to record the message as <b>consumed</b>. This strategy fixes the problem of losing messages, but creates new problems. First of all, if the consumer processes the message but fails before it can send an acknowledgement then the message will be consumed twice. The second problem is around performance, now the broker must keep multiple states about every single
  message (first to lock it so it is not given out a second time, and then to mark it as permanently consumed so that it can be removed). Tricky problems must be dealt with, like what to do with messages that are sent but never acknowledged.
+<p>
+Kafka handles this differently. Our topic is divided into a set of totally ordered partitions, each of which is consumed by one consumer at any given time. This means that the position of consumer in each partition is just a single integer, the offset of the next message to consume. This makes the state about what has been consumed very small, just one number for each partition. This state can be periodically checkpointed. This makes the equivalent of message acknowledgements very cheap.
+<p>
+There is a side benefit of this decision. A consumer can deliberately <i>rewind</i> back to an old offset and re-consume data. This violates the common contract of a queue, but turns out to be an essential feature for many consumers. For example, if the consumer code has a bug and is discovered after some messages are consumed, the consumer can re-consume those messages once the bug is fixed.
+
+<h4>Offline Data Load</h4>
+
+Scalable persistence allows for the possibility of consumers that only periodically consume such as batch data loads that periodically bulk-load data into an offline system such as Hadoop or a relational data warehouse.
+<p>
+In the case of Hadoop we parallelize the data load by splitting the load over individual map tasks, one for each node/topic/partition combination, allowing full parallelism in the loading. Hadoop provides the task management, and tasks which fail can restart without danger of duplicate data&mdash;they simply restart from their original position.
+
+<h3><a id="semantics">4.6 Message Delivery Semantics</a></h3>
+<p>
+Now that we understand a little about how producers and consumers work, let's discuss the semantic guarantees Kafka provides between producer and consumer. Clearly there are multiple possible message delivery guarantees that could be provided:
+<ul>
+  <li>
+	<i>At most once</i>&mdash;Messages may be lost but are never redelivered.
+  </li>
+  <li>
+	<i>At least once</i>&mdash;Messages are never lost but may be redelivered.
+  </li>
+  <li>
+	<i>Exactly once</i>&mdash;this is what people actually want, each message is delivered once and only once.
+  </li>
+</ul>	
+
+It's worth noting that this breaks down into two problems: the durability guarantees for publishing a message and the guarantees when consuming a message.
+<p>
+Many systems claim to provide "exactly once" delivery semantics, but it is important to read the fine print, most of these claims are misleading (i.e. they don't translate to the case where consumers or producers can fail, or cases where there are multiple consumer processes, or cases where data written to disk can be lost).
+<p>
+Kafka's semantics are straight-forward. When publishing a message we have a notion of the message being "committed" to the log. Once a published message is committed it will not be lost as long as one broker that replicates the partition to which this message was written remains "alive". The definition of alive as well as a description of which types of failures we attempt to handle will be described in more detail in the next section. For now let's assume a perfect, lossless broker and try to understand the guarantees to the producer and consumer. If a producer attempts to publish a message and experiences a network error it cannot be sure if this error happened before or after the message was committed. This is similar to the semantics of inserting into a database table with an autogenerated key.
+<p>
+These are not the strongest possible semantics for publishers. Although we cannot be sure of what happened in the case of a network error, it is possible to allow the producer to generate a sort of "primary key" that makes retrying the produce request idempotent. This feature is not trivial for a replicated system because of course it must work even (or especially) in the case of a server failure. With this feature it would suffice for the producer to retry until it receives acknowledgement of a successfully committed message at which point we would guarantee the message had been published exactly once. We hope to add this in a future Kafka version.
+<p>
+Not all use cases require such strong guarantees. For uses which are latency sensitive we allow the producer to specify the durability level it desires. If the producer specifies that it wants to wait on the message being committed this can take on the order of 10 ms. However the producer can also specify that it wants to perform the send completely asynchronously or that it wants to wait only until the leader (but not necessarily the followers) have the message.
+<p>
+Now let's describe the semantics from the point-of-view of the consumer. All replicas have the exact same log with the same offsets. The consumer controls its position in this log. If the consumer never crashed it could just store this position in memory, but if the producer fails and we want this topic partition to be taken over by another process the new process will need to choose an appropriate position from which to start processing. Let's say the consumer reads some messages -- it has several options for processing the messages and updating its position.
+<ol>
+  <li>It can read the messages, then save its position in the log, and finally process the messages. In this case there is a possibility that the consumer process crashes after saving its position but before saving the output of its message processing. In this case the process that took over processing would start at the saved position even though a few messages prior to that position had not been processed. This corresponds to "at-most-once" semantics as in the case of a consumer failure messages may not be processed.
+  <li>It can read the messages, process the messages, and finally save its position. In this case there is a possibility that the consumer process crashes after processing messages but before saving its position. In this case when the new process takes over the first few messages it receives will already have been processed. This corresponds to the "at-least-once" semantics in the case of consumer failure. In many cases messages have a primary key and so the updates are idempotent (receiving the same message twice just overwrites a record with another copy of itself).
+  <li>So what about exactly once semantics (i.e. the thing you actually want)? The limitation here is not actually a feature of the messaging system but rather the need to co-ordinate the consumer's position with what is actually stored as output. The classic way of achieving this would be to introduce a two-phase commit between the storage for the consumer position and the storage of the consumers output. But this can be handled more simply and generally by simply letting the consumer store its offset in the same place as its output. This is better because many of the output systems a consumer might want to write to will not support a two-phase commit. As an example of this, our Hadoop ETL that populates data in HDFS stores its offsets in HDFS with the data it reads so that it is guaranteed that either data and offsets are both updated or neither is. We follow similar patterns for many other data systems which require these stronger semantics and for which the messages do not have 
 a primary key to allow for deduplication.
+</ol>
+<p>
+So effectively Kafka guarantees at-least-once delivery by default and allows the user to implement at most once delivery by disabling retries on the producer and committing its offset prior to processing a batch of messages. Exactly-once delivery requires co-operation with the destination storage system but Kafka provides the offset which makes implementing this straight-forward.
+
+<h3><a id="replication">4.7 Replication</a></h3>
+<p>
+Kafka replicates the log for each topic's partitions across a configurable number of servers (you can set this replication factor on a topic-by-topic basis). This allows automatic failover to these replicas when a server in the cluster fails so messages remain available in the presence of failures.
+<p>
+Other messaging systems provide some replication-related features, but, in our (totally biased) opinion, this appears to be a tacked-on thing, not heavily used, and with large downsides: slaves are inactive, throughput is heavily impacted, it requires fiddly manual configuration, etc. Kafka is meant to be used with replication by default&mdash;in fact we implement un-replicated topics as replicated topics where the replication factor is one.
+<p>
+The unit of replication is the topic partition. Under non-failure conditions, each partition in Kafka has a single leader and zero or more followers. The total number of replicas including the leader constitute the replication factor. All reads and writes go to the leader of the partition. Typically, there are many more partitions than brokers and the leaders are evenly distributed among brokers. The logs on the followers are identical to the leader's log&mdash;all have the same offsets and messages in the same order (though, of course, at any given time the leader may have a few as-yet unreplicated messages at the end of its log).
+<p>
+Followers consume messages from the leader just as a normal Kafka consumer would and apply them to their own log. Having the followers pull from the leader has the nice property of allowing the follower to naturally batch together log entries they are applying to their log.
+<p>
+As with most distributed systems automatically handling failures requires having a precise definition of what it means for a node to be "alive". For Kafka node liveness has two conditions
+<ol>
+	<li>A node must be able to maintain its session with Zookeeper (via Zookeeper's heartbeat mechanism)
+	<li>If it is a slave it must replicate the writes happening on the leader and not fall "too far" behind
+</ol>
+We refer to nodes satisfying these two conditions as being "in sync" to avoid the vagueness of "alive" or "failed". The leader keeps track of the set of "in sync" nodes. If a follower dies, gets stuck, or falls behind, the leader will remove it from the list of in sync replicas. The definition of, how far behind is too far, is controlled by the replica.lag.max.messages configuration and the definition of a stuck replica is controlled by the replica.lag.time.max.ms configuration.
+<p>
+In distributed systems terminology we only attempt to handle a "fail/recover" model of failures where nodes suddenly cease working and then later recover (perhaps without knowing that they have died). Kafka does not handle so-called "Byzantine" failures in which nodes produce arbitrary or malicious responses (perhaps due to bugs or foul play).
+<p>
+A message is considered "committed" when all in sync replicas for that partition have applied it to their log. Only committed messages are ever given out to the consumer. This means that the consumer need not worry about potentially seeing a message that could be lost if the leader fails. Producers, on the other hand, have the option of either waiting for the message to be committed or not, depending on their preference for tradeoff between latency and durability. This preference is controlled by the request.required.acks setting that the producer uses.
+<p>
+The guarantee that Kafka offers is that a committed message will not be lost, as long as there is at least one in sync replica alive, at all times.
+<p>
+Kafka will remain available in the presence of node failures after a short fail-over period, but may not remain available in the presence of network partitions.
+
+<h4>Replicated Logs: Quorums, ISRs, and State Machines (Oh my!)</h4>
+
+At its heart a Kafka partition is a replicated log. The replicated log is one of the most basic primitives in distributed data systems, and there are many approaches for implementing one. A replicated log can be used by other systems as a primitive for implementing other distributed systems in the <a href="http://en.wikipedia.org/wiki/State_machine_replication">state-machine style</a>.
+<p>
+A replicated log models the process of coming into consensus on the order of a series of values (generally numbering the log entries 0, 1, 2, ...). There are many ways to implement this, but the simplest and fastest is with a leader who chooses the ordering of values provided to it. As long as the leader remains alive, all followers need to only copy the values and ordering, the leader chooses.
+<p>
+Of course if leaders didn't fail we wouldn't need followers! When the leader does die we need to choose a new leader from among the followers. But followers themselves may fall behind or crash so we must ensure we choose an up-to-date follower. The fundamental guarantee a log replication algorithm must provide is that if we tell the client a message is committed, and the leader fails, the new leader we elect must also have that message. This yields a tradeoff: if the leader waits for more followers to acknowledge a message before declaring it committed then there will be more potentially electable leaders.
+<p>
+If you choose the number of acknowledgements required and the number of logs that must be compared to elect a leader such that there is guaranteed to be an overlap, then this is called a Quorum.
+<p>
+A common approach to this tradeoff is to use a majority vote for both the commit decision and the leader election. This is not what Kafka does, but let's explore it anyway to understand the tradeoffs. Let's say we have 2<i>f</i>+1 replicas. If <i>f</i>+1 replicas must receive a message prior to a commit being declared by the leader, and if we elect a new leader by electing the follower with the most complete log from at least <i>f</i>+1 replicas, then, with no more than <i>f</i> failures, the leader is guaranteed to have all committed messages. This is because among any <i>f</i>+1 replicas, there must be at least one replica that contains all committed messages. That replica's log will be the most complete and therefore will be selected as the new leader. There are many remaining details that each algorithm must handle (such as precisely defined what makes a log more complete, ensuring log consistency during leader failure or changing the set of servers in the replica set) but we wi
 ll ignore these for now.
+<p>
+This majority vote approach has a very nice property: the latency is dependent on only the fastest servers. That is, if the replication factor is three, the latency is determined by the faster slave not the slower one.
+<p>
+There are a rich variety of algorithms in this family including Zookeeper's <a href="http://www.stanford.edu/class/cs347/reading/zab.pdf">Zab</a>, <a href="https://ramcloud.stanford.edu/wiki/download/attachments/11370504/raft.pdf">Raft</a>, and <a href="http://pmg.csail.mit.edu/papers/vr-revisited.pdf">Viewstamped Replication</a>. The most similar academic publication we are aware of to Kafka's actual implementation is <a href="http://research.microsoft.com/apps/pubs/default.aspx?id=66814">PacificA</a> from Microsoft.
+<p>
+The downside of majority vote is that it doesn't take many failures to leave you with no electable leaders. To tolerate one failure requires three copies of the data, and to tolerate two failures requires five copies of the data. In our experience having only enough redundancy to tolerate a single failure is not enough for a practical system, but doing every write five times, with 5x the disk space requirements and 1/5th the throughput, is not very practical for large volume data problems. This is likely why quorum algorithms more commonly appear for shared cluster configuration such as Zookeeper but are less common for primary data storage. For example in HDFS the namenode's high-availability feature is built on a <a href="http://blog.cloudera.com/blog/2012/10/quorum-based-journaling-in-cdh4-1">majority-vote-based journal</a>, but this more expensive approach is not used for the data itself.
+<p>
+Kafka takes a slightly different approach to choosing its quorum set. Instead of majority vote, Kafka dynamically maintains a set of in-sync replicas (ISR) that are caught-up to the leader. Only members of this set are eligible for election as leader. A write to a Kafka partition is not considered committed until <i>all</i> in-sync replicas have received the write. This ISR set is persisted to zookeeper whenever it changes. Because of this, any replica in the ISR is eligible to be elected leader. This is an important factor for Kafka's usage model where there are many partitions and ensuring leadership balance is important. With this ISR model and <i>f+1</i> replicas, a Kafka topic can tolerate <i>f</i> failures without losing committed messages.
+<p>
+For most use cases we hope to handle, we think this tradeoff is a reasonable one. In practice, to tolerate <i>f</i> failures, both the majority vote and the ISR approach will wait for the same number of replicas to acknowledge before committing a message (e.g. to survive one failure a majority quorum needs three replicas and one acknowledgement and the ISR approach requires two replicas and one acknowledgement). The ability to commit without the slowest servers is an advantage of the majority vote approach. However, we think it is ameliorated by allowing the client to choose whether they block on the message commit or not, and the additional throughput and disk space due to the lower required replication factor is worth it.
+<p>
+Another important design distinction is that Kafka does not require that crashed nodes recover with all their data intact. It is not uncommon for replication algorithms in this space to depend on the existence of "stable storage" that cannot be lost in any failure-recovery scenario without potential consistency violations. There are two primary problems with this assumption. First, disk errors are the most common problem we observe in real operation of persistent data systems and they often do not leave data intact. Secondly, even if this were not a problem, we do not want to require the use of fsync on every write for our consistency guarantees as this can reduce performance by two to three orders of magnitude. Our protocol for allowing a replica to rejoin the ISR ensures that before rejoining, it must fully re-sync again even if it lost unflushed data in its crash.
+
+<h4>Unclean leader election: What if they all die?</h4>
+
+Note that Kafka's guarantee with respect to data loss is predicated on at least on replica remaining in sync. If the current leader dies and no remaining live replicas are in the ISR, this guarantee no longer holds. If your have more than one replica assigned to a partiiton, this should be relatively rare since at least two brokers have to fail for this to happen.
+<p>
+However a practical system needs to do something reasonable when all in-sync replicas die. If you are unlucky enough to have this occur, it is important to consider what will happen. There are two behaviors that could be implemented:
+<ol>
+	<li>Wait for a replica in the ISR to come back to life and choose this replica as the leader (hopefully it still has all its data).
+	<li>Choose the first replica (not necessarily in the ISR) that comes back to life as the leader.
+</ol>
+<p>
+This is a simple tradeoff between availability and consistency. If we wait for replicas in the ISR, then we will remain unavailable as long as those replicas are down. If such replicas were destroyed or their data was lost, then we are permanently down. If, on the other hand, a non-in-sync replica comes back to life and we allow it to become the leader, then its log becomes the source of truth even though it is not guaranteed to have every committed message. In our current release we choose the second strategy and favor choosing a potentially inconsistent replica when all replicas in the ISR are dead. In the future, we would like to make this configurable to better support use cases where downtime is preferable to inconsistency.
+<p>
+This dilemma is not specific to Kafka. It exists in any quorum-based scheme. For example in a majority voting scheme, if a majority of servers suffer a permanent failure, then you must either choose to lose 100% of your data or violate consistency by taking what remains on an existing server as your new source of truth.
+
+<h4>Replica Management</h4>
+
+The above discussion on replicated logs really covers only a single log, i.e. one topic partition. However a Kafka cluster will manage hundreds or thousands of these partitions. We attempt to balance partitions within a cluster in a round-robin fashion to avoid clustering all partitions for high-volume topics on a small number of nodes. Likewise we try to balance leadership so that each node is the leader for a proportional share of its partitions.
+<p>
+It is also important to optimize the leadership election process as that is the critical window of unavailability. A naive implementation of leader election would end up running an election per partition for all partitions a node hosted when that node failed. Instead, we elect one of the brokers as the "controller". This controller detects failures at the broker level and is responsible for changing the leader of all affected partitions in a failed broker. The result is that we are able to batch together many of the required leadership change notifications which makes the election process far cheaper and faster for a large number of partitions. If the controller fails, one of the surviving brokers will become the new controller.

http://git-wip-us.apache.org/repos/asf/kafka-site/blob/ba6c994c/08/documentation.html
----------------------------------------------------------------------
diff --git a/08/documentation.html b/08/documentation.html
new file mode 100644
index 0000000..6720754
--- /dev/null
+++ b/08/documentation.html
@@ -0,0 +1,101 @@
+<!--#include virtual="../includes/header.html" -->
+
+<h1>Kafka 0.8.0 Documentation</h1>
+<i>Documentation for the 0.7 release is <a href="/07/documentation.html">here</a>.</i>
+<ul class="toc">
+    <li><a href="#gettingStarted">1. Getting Started</a>
+         <ul>
+             <li><a href="#introduction">1.1 Introduction</a>
+	         <li><a href="#uses">1.2 Use Cases</a>
+             <li><a href="#quickstart">1.3 Quick Start</a>
+         </ul>
+    <li><a href="#api">2. API</a>
+	      <ul>
+		      <li><a href="#producerapi">2.1 Producer API</a>
+			  <li><a href="#highlevelconsumerapi">2.2 High Level Consumer API</a>
+			  <li><a href="#simpleconsumerapi">2.3 Simple Consumer API</a>
+			  <li><a href="#kafkahadoopconsumerapi">2.4 Kafka Hadoop Consumer API</a>
+		  </ul>
+    <li><a href="#configuration">3. Configuration</a>
+	    <ul>
+		     <li><a href="#brokerconfigs">3.1 Broker Configs</a>
+			 <li><a href="#consumerconfigs">3.2 Consumer Configs</a>
+		     <li><a href="#producerconfigs">3.3 Producer Configs</a>
+		</ul>
+    <li><a href="#design">4. Design</a>
+	    <ul>
+		     <li><a href="#majordesignelements">4.1 Motivation</a>
+			 <li><a href="#persistence">4.2 Persistence</a>
+			 <li><a href="#maximizingefficiency">4.3 Efficiency</a>
+			 <li><a href="#theproducer">4.4 The Producer</a>
+			 <li><a href="#theconsumer">4.5 The Consumer</a>
+			 <li><a href="#semantics">4.6 Message Delivery Semantics</a>
+			 <li><a href="#replication">4.7 Replication</a>
+		</ul>
+	<li><a href="#implementation">5. Implementation</a>
+		<ul>
+			  <li><a href="#apidesign">5.1 API Design</a>
+			  <li><a href="#networklayer">5.2 Network Layer</a>
+			  <li><a href="#messages">5.3 Messages</a>
+			  <li><a href="#messageformat">5.4 Message format</a>
+			  <li><a href="#log">5.5 Log</a>
+			  <li><a href="#distributionimpl">5.6 Distribution</a>
+		</ul>
+	<li><a href="#operations">6. Operations</a>
+		<ul>
+			  <li><a href="#datacenters">6.1 Datacenters</a>
+			  <li><a href="#config">6.2 Config</a>
+				 <ul>
+					 <li><a href="#serverconfig">Important Server Configs</a>
+					 <li><a href="#clientconfig">Important Client Configs</a>
+					 <li><a href="#prodconfig">A Production Server Configs</a>
+        		 </ul>
+     		  <li><a href="#java">6.3 Java Version</a>
+	 		  <li><a href="#hwandos">6.4 Hardware and OS</a>
+				<ul>
+					<li><a href="#os">OS</a>
+					<li><a href="#diskandfs">Disks and Filesystems</a>
+					<li><a href="#appvsosflush">Application vs OS Flush Management</a>
+					<li><a href="#linuxflush">Linux Flush Behavior</a>
+					<li><a href="#ext4">Ext4 Notes</a>
+				</ul>
+			  <li><a href="#monitoring">6.5 Monitoring</a>
+			  <li><a href="#zk">6.6 Zookeeper</a>
+				<ul>
+					<li><a href="#zkversion">Stable Version</a>
+					<li><a href="#zkops">Operationalization</a>
+				</ul>
+		</ul>
+	<li><a href="#tools">7. Tools</a>
+</ul>
+
+<h2><a id="gettingStarted">1. Getting Started</a></h2>
+<!--#include virtual="introduction.html" -->
+<!--#include virtual="uses.html" -->
+<!--#include virtual="quickstart.html" -->
+
+<h2><a id="api">2. API</a></h2>
+
+<!--#include virtual="api.html" -->
+
+<h2><a id="configuration">3. Configuration</a></h2>
+
+<!--#include virtual="configuration.html" -->
+
+<h2><a id="design">4. Design</a></h2>
+
+<!--#include virtual="design.html" -->
+
+<h2><a id="implementation">5. Implementation</a></h2>
+
+<!--#include virtual="implementation.html" -->
+
+<h2><a id="operations">6. Operations</a></h2>
+
+<!--#include virtual="ops.html" -->
+
+<h2><a id="tools">7. Tools</a></h2>
+
+<!--#include virtual="tools.html" -->
+
+<!--#include virtual="../includes/footer.html" -->


Mime
View raw message