You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kafka.apache.org by jk...@apache.org on 2014/03/05 22:06:37 UTC

svn commit: r1574659 [2/5] - in /kafka/site: 08/ 081/ diagrams/ images/

Added: kafka/site/081/design.html
URL: http://svn.apache.org/viewvc/kafka/site/081/design.html?rev=1574659&view=auto
==============================================================================
--- kafka/site/081/design.html (added)
+++ kafka/site/081/design.html Wed Mar  5 21:06:36 2014
@@ -0,0 +1,304 @@
+<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 new the system would have to be able to guarantee fault-tolerance in the presence of machine failures.
+<p>
+Supporting these uses led use 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 it's 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 consumers 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 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 pri
 mary 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 gives 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 it's 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 all the nodes replicating a partition die, this guarantee no longer holds.
+<p>
+However a practical system needs to do something reasonable when all the 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 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.
+
+<h3><a id="compaction">4.8 Log Compaction</a></h3>
+
+One of the ways Kafka is different from traditional messaging systems is that it maintains historical log data beyond just the currently in-flight messages. This allows a set of usage patterns and system architectures that make use of this as a kind of external commit log. Log compaction is a feature that helps to support this kind of use case.
+<p>
+So far we have described only the simpler approach to data retention where old log data is discarded after a fixed period of time or when the log reaches some predetermined size. This works well for temporal event data such as logging where each record stands alone. However an important class of data streams are the log of changes to keyed, mutable data (for example, the changes to a database). For these data streams maintaining only the most recent changes means that the log will have only a subset of the full data set, which limits its usefulness. However maintaining the full log of all updates for all time would require an unbounded amount of space. Log compaction addresses this issue by adding a data retention mechanism that allows pruning obsolete values by primary key rather than requiring that full segments be discarded all together.
+<p>
+Let's start with a few examples of use cases that log updates, then we'll talk about how Kafka's log compaction supports these use cases.
+<ol>
+<li><i>Database change subscription</i>. It is often necessary to have a data set in multiple data systems, and often one of these systems is a database of some kind (either a RDBMS or perhaps a newfangled key-value store). For example you might have a database, a cache, a search cluster, and a Hadoop cluster. Each change to the database will need to be reflected in the cache, the search cluster, and eventually in Hadoop. In the case that one is only handling the real-time updates 
+<li><i>Event sourcing</i>. This is a style of application design which co-locates query processing with application design and uses a log of changes as the primary store for the application.
+<li><i>Journaling for high-availability</i>. A process that does local computation can be made fault-tolerant by logging out changes that it makes to it's local state so another process can reload these changes and carry on if it should fail. A concrete example of this is handling counts, aggregations, and other "group by"-like processing in a stream query system. Samza, a real-time stream-processing framework, <a href="http://samza.incubator.apache.org/learn/documentation/0.7.0/container/state-management.html">uses this feature</a> for exactly this purpose.
+</ol>
+In each of these cases one needs primarily to handle the real-time feed of changes, but occasionally, when a machine crashes or data needs to be re-loaded or re-processed, one needs to do a full load. Log compaction allows feeding both of these use cases off the same backing topic.
+
+This style of usage of a log is described in more detail in <a href="http://engineering.linkedin.com/distributed-systems/log-what-every-software-engineer-should-know-about-real-time-datas-unifying">this blog post</a>.
+<p>
+The general idea is quite simple. If we had infinite log retention, and we logged each change in the above cases, then we would have captured the state of the system at each time from when it first began. Using this complete log we could restore to any point in time by replaying the first N records in the log. This hypothetical complete log is not very practical for systems that update a single record many times as the log will grow without bound even for a stable dataset. The simple log retention mechanism which throws away old updates will bound space but the log is no longer a way to restore the current state&mdash;now restoring from the beginning of the log no longer recreates the current state as old updates may not be captured at all.
+<p>
+Log compaction is a mechanism to give finer-grained per-record retention, rather than the coarser-grained time-based retention. The idea is to selectively remove records where we have a more recent update with the same primary key. This way the log is guaranteed to have at least the last state for each key.
+<p>
+This retention policy can be set per-topic, so topics retained by time can mix with topics which are compacted.
+<p> 
+This functionality is inspired by one of LinkedIn's oldest and most successful pieces of infrastructure&mdash;a database changelog caching service called <a href="https://github.com/linkedin/databus">Databus</a>. Unlike most log-structured storage systems Kafka is built for subscription and organizes data for fast linear reads and writes. Unlike Databus, Kafka acts a source-of-truth store so it is useful even in situations where the upstream data source would not otherwise be replayable.
+
+<h4>Log Compaction Basics</h4>
+
+Here is a high-level picture that shows the logical structure of a Kafka log with the offset for each message.
+<img src="/images/log_cleaner_anatomy.png">
+<p>
+The head of the log is identical to a traditional Kafka log. It has dense, sequential offsets and retains all messages. Log compaction adds an option for handling the tail of the log. The picture above shows a log with a compacted tail. Note that the messages in the tail of the log retain the original offset assigned when they were first written&mdash;that never changes. Note also that all offsets remain valid positions in the log, even if the message with that offset has been compacted away; in this case this position is indistinguishable from the next highest offset that does appear in the log. For example, in the picture above the offsets 36, 37, and 38 are all equivalent positions and a read beginning at any of these offsets would return a message set beginning with 38.
+<p>
+Compaction also allows for deletes. A message with a key and a null payload will be treated as a delete from the log. This delete marker will cause any prior message with that key to be removed (as would any new message with that key), but delete markers are special in they will themselves be cleaned out of the log after a period of time to free up space. The point in time at which deletes are no longer retained is marked as the "delete retention point" in the above diagram.
+<p>
+The compaction is done in the background by periodically recopying log segments. Cleaning does not block reads and can be throttled to use no more than a configurable amount of I/O throughput to avoid impacting producers and consumers. The actual process of compacting a log segment looks something like this:
+<img src="/images/log_compaction.png">
+<p>
+<h4>What guarantees does log compaction provide?</h4>
+
+Log compaction guarantees the following:
+<ol>
+<li>Any consumer that stays caught-up to within the head of the log will see a message for each offset.
+<li>Message order is always maintained (that is, compaction will never re-order messages just remove some).
+<li>The offset for a message never changes, it is the permanent identifier for a position in the log.
+<li>Any read from offset 0 will contain see the final state of all records provided it completes in less than a configurable time period (even if further compaction occurs during this read).
+</ol>
+The third guarantee seems somewhat esoteric but this ensures that a reader will end in a coherent state (equivalent to reading the full uncompacted log) even though the underlying log may be compacted while this read occurs.
+
+<h4>Log Compaction Details</h4>
+
+Log compaction is handled by the log cleaner, a pool of background threads that recopy log segment files, removing records whose key appears in the head of the log. Each compactor thread works as follows:
+<ol>
+<li>It chooses the log that has the highest ratio of log head to log tail
+<li>It creates a succinct summary of the last offset for each key in the head of the log
+<li>It recopies the log from beginning to end removing keys which have a later occurrence in the log. New, clean segments are swapped into the log immediately so the additional disk space required is just one additional log segment (not a fully copy of the log).
+<li>The summary of the log head is essentially just a space-compact hash table. It uses exactly 24 bytes per entry. As a result with 8GB of cleaner buffer one cleaner iteration can clean around 366GB of log head (assuming 1k messages).
+</ol>
+<p>
+<h4>Configuring The Log Cleaner</h4>
+
+As of 0.8.1 the log cleaner is disabled by default. To enable it set the server config
+  <pre>  log.cleaner.enable=true</pre>
+This will start the pool of cleaner threads. To enable log cleaning on a particular topic you can add the log-specific property
+  <pre>  log.cleanup.policy=compact</pre>
+This can be done either at topic creation time or using the alter topic command.
+<p>
+Further cleaner configurations are described <a href="/documentation.html#brokerconfigs">here</a>.
+
+<h4>Limitations</h4>
+We haven't yet made it configurable how much log is retained without compaction (the "head" of the log), so currently all segments are eligible except for the last segment (the one currently being written to).
+<p>
+Log compaction is also not yet compatible with compression.
\ No newline at end of file

Added: kafka/site/081/documentation.html
URL: http://svn.apache.org/viewvc/kafka/site/081/documentation.html?rev=1574659&view=auto
==============================================================================
--- kafka/site/081/documentation.html (added)
+++ kafka/site/081/documentation.html Wed Mar  5 21:06:36 2014
@@ -0,0 +1,106 @@
+<!--#include virtual="../includes/header.html" -->
+
+<h1>Kafka 0.8.1 Documentation</h1>
+Prior releases: <a href="/07/documentation.html">0.7.x</a>, <a href="/08/documentation.html">0.8.0</a>.
+</ul>
+	
+<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>
+	         <li><a href="#upgrade">1.4 Upgrading</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>
+			 <li><a href="#compaction">4.8 Log Compaction</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" -->
+<!--#include virtual="upgrade.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" -->

Added: kafka/site/081/implementation.html
URL: http://svn.apache.org/viewvc/kafka/site/081/implementation.html?rev=1574659&view=auto
==============================================================================
--- kafka/site/081/implementation.html (added)
+++ kafka/site/081/implementation.html Wed Mar  5 21:06:36 2014
@@ -0,0 +1,346 @@
+<h3><a id="apidesign">5.1 API Design</a></h3>
+
+<h4>Producer APIs</h4>
+
+<p>
+The Producer API that wraps the 2 low-level producers - <code>kafka.producer.SyncProducer</code> and <code>kafka.producer.async.AsyncProducer</code>. 
+<pre>
+class Producer<T> {
+	
+  /* Sends the data, partitioned by key to the topic using either the */
+  /* synchronous or the asynchronous producer */
+  public void send(kafka.javaapi.producer.ProducerData&lt;K,V&gt; producerData);
+
+  /* Sends a list of data, partitioned by key to the topic using either */
+  /* the synchronous or the asynchronous producer */
+  public void send(java.util.List&lt;kafka.javaapi.producer.ProducerData&lt;K,V&gt;&gt; producerData);
+
+  /* Closes the producer and cleans up */	
+  public void close();
+
+}
+</pre>
+
+The goal is to expose all the producer functionality through a single API to the client.  
+
+The new producer -
+<ul>
+<li>can handle queueing/buffering of multiple producer requests and asynchronous dispatch of the batched data - 	
+<p><code>kafka.producer.Producer</code> provides the ability to batch multiple produce requests (<code>producer.type=async</code>), before serializing and dispatching them to the appropriate kafka broker partition. The size of the batch can be controlled by a few config parameters. As events enter a queue, they are buffered in a queue, until either <code>queue.time</code> or <code>batch.size</code> is reached. A background thread (<code>kafka.producer.async.ProducerSendThread</code>) dequeues the batch of data and lets the <code>kafka.producer.EventHandler</code> serialize and send the data to the appropriate kafka broker partition. A custom event handler can be plugged in through the <code>event.handler</code> config parameter. At various stages of this producer queue pipeline, it is helpful to be able to inject callbacks, either for plugging in custom logging/tracing code or custom monitoring logic. This is possible by implementing the <code>kafka.producer.async.CallbackHandler</c
 ode> interface and setting <code>callback.handler</code> config parameter to that class.
+</p>
+</li>
+<li>handles the serialization of data through a user-specified <code>Encoder</code> - 
+<pre>
+interface Encoder&lt;T&gt; {
+  public Message toMessage(T data);
+}
+</pre>
+<p>The default is the no-op <code>kafka.serializer.DefaultEncoder</code></p>
+</li>
+<li>provides zookeeper based automatic broker discovery - 
+<p>
+The zookeeper based broker discovery and load balancing can be used by specifying the zookeeper connection url through the <code>zk.connect</code> config parameter. For some applications, however, the dependence on zookeeper is inappropriate. In that case, the producer can take in a static list of brokers through the <code>broker.list</code> config parameter. Each produce requests gets routed to a random broker partition in this case. If that broker is down, the produce request fails. 
+</p>
+</li>
+<li>provides software load balancing through an optionally user-specified <code>Partitioner</code> - 
+<p>
+The routing decision is influenced by the <code>kafka.producer.Partitioner</code>. 
+<pre>
+interface Partitioner&lt;T&gt; {
+   int partition(T key, int numPartitions);
+}
+</pre>
+The partition API uses the key and the number of available broker partitions to return a partition id. This id is used as an index into a sorted list of broker_ids and partitions to pick a broker partition for the producer request. The default partitioning strategy is <code>hash(key)%numPartitions</code>. If the key is null, then a random broker partition is picked. A custom partitioning strategy can also be plugged in using the <code>partitioner.class</code> config parameter.	
+</p>
+</li>
+</ul>
+</p>
+
+<h4>Consumer APIs</h4>
+<p>
+We have 2 levels of consumer APIs. The low-level "simple" API maintains a connection to a single broker and has a close correspondence to the network requests sent to the server. This API is completely stateless, with the offset being passed in on every request, allowing the user to maintain this metadata however they choose.
+</p>
+<p>
+The high-level API hides the details of brokers from the consumer and allows consuming off the cluster of machines without concern for the underlying topology. It also maintains the state of what has been consumed. The high-level API also provides the ability to subscribe to topics that match a filter expression (i.e., either a whitelist or a blacklist regular expression).
+</p>
+
+<h5>Low-level API</h5>
+<pre>
+class SimpleConsumer {
+	
+  /* Send fetch request to a broker and get back a set of messages. */ 
+  public ByteBufferMessageSet fetch(FetchRequest request);
+
+  /* Send a list of fetch requests to a broker and get back a response set. */ 
+  public MultiFetchResponse multifetch(List&lt;FetchRequest&gt; fetches);
+
+  /**
+   * Get a list of valid offsets (up to maxSize) before the given time.
+   * The result is a list of offsets, in descending order.
+   * @param time: time in millisecs,
+   *              if set to OffsetRequest$.MODULE$.LATIEST_TIME(), get from the latest offset available.
+   *              if set to OffsetRequest$.MODULE$.EARLIEST_TIME(), get from the earliest offset available.
+   */
+  public long[] getOffsetsBefore(String topic, int partition, long time, int maxNumOffsets);
+}
+</pre>
+
+The low-level API is used to implement the high-level API as well as being used directly for some of our offline consumers (such as the hadoop consumer) which have particular requirements around maintaining state.
+
+<h5>High-level API</h5>
+<pre>
+
+/* create a connection to the cluster */ 
+ConsumerConnector connector = Consumer.create(consumerConfig);
+
+interface ConsumerConnector {
+	
+  /**
+   * This method is used to get a list of KafkaStreams, which are iterators over
+   * MessageAndMetadata objects from which you can obtain messages and their
+   * associated metadata (currently only topic).
+   *  Input: a map of &lt;topic, #streams&gt;
+   *  Output: a map of &lt;topic, list of message streams&gt;
+   */
+  public Map&lt;String,List&lt;KafkaStream&gt;&gt; createMessageStreams(Map&lt;String,Int&gt; topicCountMap); 
+
+  /**
+   * You can also obtain a list of KafkaStreams, that iterate over messages
+   * from topics that match a TopicFilter. (A TopicFilter encapsulates a
+   * whitelist or a blacklist which is a standard Java regex.)
+   */
+  public List&lt;KafkaStream&gt; createMessageStreamsByFilter(
+      TopicFilter topicFilter, int numStreams);
+
+  /* Commit the offsets of all messages consumed so far. */
+  public commitOffsets()
+  
+  /* Shut down the connector */
+  public shutdown()
+}
+</pre>
+<p>
+This API is centered around iterators, implemented by the KafkaStream class. Each KafkaStream represents the stream of messages from one or more partitions on one or more servers. Each stream is used for single threaded processing, so the client can provide the number of desired streams in the create call. Thus a stream may represent the merging of multiple server partitions (to correspond to the number of processing threads), but each partition only goes to one stream.
+</p>
+<p>
+The createMessageStreams call registers the consumer for the topic, which results in rebalancing the consumer/broker assignment. The API encourages creating many topic streams in a single call in order to minimize this rebalancing. The createMessageStreamsByFilter call (additionally) registers watchers to discover new topics that match its filter. Note that each stream that createMessageStreamsByFilter returns may iterate over messages from multiple topics (i.e., if multiple topics are allowed by the filter).
+</p>
+
+<h3><a id="networklayer">5.2 Network Layer</a></h3>
+<p>
+The network layer is a fairly straight-forward NIO server, and will not be described in great detail. The sendfile implementation is done by giving the <code>MessageSet</code> interface a <code>writeTo</code> method. This allows the file-backed message set to use the more efficient <code>transferTo</code> implementation instead of an in-process buffered write. The threading model is a single acceptor thread and <i>N</i> processor threads which handle a fixed number of connections each. This design has been pretty thoroughly tested <a href="http://sna-projects.com/blog/2009/08/introducing-the-nio-socketserver-implementation">elsewhere</a> and found to be simple to implement and fast. The protocol is kept quite simple to allow for future implementation of clients in other languages.
+</p>
+<h3><a id="messages">5.3 Messages</a></h3>
+<p>
+Messages consist of a fixed-size header and variable length opaque byte array payload. The header contains a format version and a CRC32 checksum to detect corruption or truncation. Leaving the payload opaque is the right decision: there is a great deal of progress being made on serialization libraries right now, and any particular choice is unlikely to be right for all uses. Needless to say a particular application using Kafka would likely mandate a particular serialization type as part of its usage. The <code>MessageSet</code> interface is simply an iterator over messages with specialized methods for bulk reading and writing to an NIO <code>Channel</code>.
+
+<h3><a id="messageformat">5.4 Message Format</a></h3>
+
+<pre>
+	/** 
+	 * A message. The format of an N byte message is the following: 
+	 * 
+	 * If magic byte is 0 
+	 * 
+	 * 1. 1 byte "magic" identifier to allow format changes 
+	 * 
+	 * 2. 4 byte CRC32 of the payload 
+	 * 
+	 * 3. N - 5 byte payload 
+	 * 
+	 * If magic byte is 1 
+	 * 
+	 * 1. 1 byte "magic" identifier to allow format changes 
+	 * 
+	 * 2. 1 byte "attributes" identifier to allow annotations on the message independent of the version (e.g. compression enabled, type of codec used) 
+	 * 
+	 * 3. 4 byte CRC32 of the payload 
+	 * 
+	 * 4. N - 6 byte payload 
+	 * 
+	 */
+</pre>
+</p>
+<h3><a id="log">5.5 Log</a></h3>
+<p>
+A log for a topic named "my_topic" with two partitions consists of two directories (namely <code>my_topic_0</code> and <code>my_topic_1</code>) populated with data files containing the messages for that topic. The format of the log files is a sequence of "log entries""; each log entry is a 4 byte integer <i>N</i> storing the message length which is followed by the <i>N</i> message bytes. Each message is uniquely identified by a 64-bit integer <i>offset</i> giving the byte position of the start of this message in the stream of all messages ever sent to that topic on that partition. The on-disk format of each message is given below. Each log file is named with the offset of the first message it contains. So the first file created will be 00000000000.kafka, and each additional file will have an integer name roughly <i>S</i> bytes from the previous file where <i>S</i> is the max log file size given in the configuration.
+</p>
+<p>
+The exact binary format for messages is versioned and maintained as a standard interface so message sets can be transfered between producer, broker, and client without recopying or conversion when desirable. This format is as follows:
+</p>
+<pre>
+On-disk format of a message
+
+message length : 4 bytes (value: 1+4+n) 
+"magic" value  : 1 byte
+crc            : 4 bytes
+payload        : n bytes
+</pre>
+<p>
+The use of the message offset as the message id is unusual. Our original idea was to use a GUID generated by the producer, and maintain a mapping from GUID to offset on each broker. But since a consumer must maintain an ID for each server, the global uniqueness of the GUID provides no value. Furthermore the complexity of maintaining the mapping from a random id to an offset requires a heavy weight index structure which must be synchronized with disk, essentially requiring a full persistent random-access data structure. Thus to simplify the lookup structure we decided to use a simple per-partition atomic counter which could be coupled with the partition id and node id to uniquely identify a message; this makes the lookup structure simpler, though multiple seeks per consumer request are still likely. However once we settled on a counter, the jump to directly using the offset seemed natural&mdash;both after all are monotonically increasing integers unique to a partition. Since the offs
 et is hidden from the consumer API this decision is ultimately an implementation detail and we went with the more efficient approach.
+</p>
+<img src="../images/kafka_log.png">
+<h4>Writes</h4>
+<p>
+The log allows serial appends which always go to the last file. This file is rolled over to a fresh file when it reaches a configurable size (say 1GB). The log takes two configuration parameter <i>M</i> which gives the number of messages to write before forcing the OS to flush the file to disk, and <i>S</i> which gives a number of seconds after which a flush is forced. This gives a durability guarantee of losing at most <i>M</i> messages or <i>S</i> seconds of data in the event of a system crash.
+</p>
+<h4>Reads</h4>
+<p>
+Reads are done by giving the 64-bit logical offset of a message and an <i>S</i>-byte max chunk size. This will return an iterator over the messages contained in the <i>S</i>-byte buffer. <i>S</i> is intended to be larger than any single message, but in the event of an abnormally large message, the read can be retried multiple times, each time doubling the buffer size, until the message is read successfully. A maximum message and buffer size can be specified to make the server reject messages larger than some size, and to give a bound to the client on the maximum it need ever read to get a complete message. It is likely that the read buffer ends with a partial message, this is easily detected by the size delimiting.
+</p>
+<p>
+The actual process of reading from an offset requires first locating the log segment file in which the data is stored, calculating the file-specific offset from the global offset value, and then reading from that file offset. The search is done as a simple binary search variation against an in-memory range maintained for each file.
+</p>
+<p>
+The log provides the capability of getting the most recently written message to allow clients to start subscribing as of "right now". This is also useful in the case the consumer fails to consume its data within its SLA-specified number of days. In this case when the client attempts to consume a non-existant offset it is given an OutOfRangeException and can either reset itself or fail as appropriate to the use case.
+</p>
+
+<p> The following is the format of the results sent to the consumer.
+
+<pre>
+MessageSetSend (fetch result)
+
+total length     : 4 bytes
+error code       : 2 bytes
+message 1        : x bytes
+...
+message n        : x bytes
+</pre>
+
+<pre>
+MultiMessageSetSend (multiFetch result)
+
+total length       : 4 bytes
+error code         : 2 bytes
+messageSetSend 1
+...
+messageSetSend n
+</pre>
+
+<h4>Deletes</h4>
+<p>
+Data is deleted one log segment at a time. The log manager allows pluggable delete policies to choose which files are eligible for deletion. The current policy deletes any log with a modification time of more than <i>N</i> days ago, though a policy which retained the last <i>N</i> GB could also be useful. To avoid locking reads while still allowing deletes that modify the segment list we use a copy-on-write style segment list implementation that provides consistent views to allow a binary search to proceed on an immutable static snapshot view of the log segments while deletes are progressing.
+</p>
+<h4>Guarantees</h4>
+<p>
+The log provides a configuration parameter <i>M</i> which controls the maximum number of messages that are written before forcing a flush to disk. On startup a log recovery process is run that iterates over all messages in the newest log segment and verifies that each message entry is valid. A message entry is valid if the sum of its size and offset are less than the length of the file AND the CRC32 of the message payload matches the CRC stored with the message. In the event corruption is detected the log is truncated to the last valid offset.
+</p>
+<p>
+Note that two kinds of corruption must be handled: truncation in which an unwritten block is lost due to a crash, and corruption in which a nonsense block is ADDED to the file. The reason for this is that in general the OS makes no guarantee of the write order between the file inode and the actual block data so in addition to losing written data the file can gain nonsense data if the inode is updated with a new size but a crash occurs before the block containing that data is not written. The CRC detects this corner case, and prevents it from corrupting the log (though the unwritten messages are, of course, lost).
+</p>
+
+<h3><a id="distributionimpl">5.6 Distribution</a></h3>
+<h4>Zookeeper Directories</h4>
+<p>
+The following gives the zookeeper structures and algorithms used for co-ordination between consumers and brokers.
+</p>
+
+<h4>Notation</h4>
+<p>
+When an element in a path is denoted [xyz], that means that the value of xyz is not fixed and there is in fact a zookeeper znode for each possible value of xyz. For example /topics/[topic] would be a directory named /topics containing a sub-directory for each topic name. Numerical ranges are also given such as [0...5] to indicate the subdirectories 0, 1, 2, 3, 4. An arrow -> is used to indicate the contents of a znode. For example /hello -> world would indicate a znode /hello containing the value "world".
+</p>
+
+<h4>Broker Node Registry</h4>
+<pre>
+/brokers/ids/[0...N] --> host:port (ephemeral node)
+</pre>
+<p>
+This is a list of all present broker nodes, each of which provides a unique logical broker id which identifies it to consumers (which must be given as part of its configuration). On startup, a broker node registers itself by creating a znode with the logical broker id under /brokers/ids. The purpose of the logical broker id is to allow a broker to be moved to a different physical machine without affecting consumers. An attempt to register a broker id that is already in use (say because two servers are configured with the same broker id) is an error.
+</p>
+<p>
+Since the broker registers itself in zookeeper using ephemeral znodes, this registration is dynamic and will disappear if the broker is shutdown or dies (thus notifying consumers it is no longer available).	
+</p>
+<h4>Broker Topic Registry</h4>
+<pre>
+/brokers/topics/[topic]/[0...N] --> nPartions (ephemeral node)
+</pre>
+
+<p>
+Each broker registers itself under the topics it maintains and stores the number of partitions for that topic.
+</p>
+
+<h4>Consumers and Consumer Groups</h4>
+<p>
+Consumers of topics also register themselves in Zookeeper, in order to balance the consumption of data and track their offsets in each partition for each broker they consume from.
+</p>
+
+<p>
+Multiple consumers can form a group and jointly consume a single topic. Each consumer in the same group is given a shared group_id. 
+For example if one consumer is your foobar process, which is run across three machines, then you might assign this group of consumers the id "foobar". This group id is provided in the configuration of the consumer, and is your way to tell the consumer which group it belongs to.
+</p>
+
+<p>
+The consumers in a group divide up the partitions as fairly as possible, each partition is consumed by exactly one consumer in a consumer group.
+</p>
+
+<h4>Consumer Id Registry</h4>
+<p>
+In addition to the group_id which is shared by all consumers in a group, each consumer is given a transient, unique consumer_id (of the form hostname:uuid) for identification purposes. Consumer ids are registered in the following directory.
+<pre>
+/consumers/[group_id]/ids/[consumer_id] --> {"topic1": #streams, ..., "topicN": #streams} (ephemeral node)
+</pre>
+Each of the consumers in the group registers under its group and creates a znode with its consumer_id. The value of the znode contains a map of &lt;topic, #streams&gt;. This id is simply used to identify each of the consumers which is currently active within a group. This is an ephemeral node so it will disappear if the consumer process dies.
+</p>
+
+<h4>Consumer Offset Tracking</h4>
+<p>
+Consumers track the maximum offset they have consumed in each partition. This value is stored in a zookeeper directory
+</p>
+<pre>
+/consumers/[group_id]/offsets/[topic]/[broker_id-partition_id] --> offset_counter_value ((persistent node)
+</pre>
+
+<h4>Partition Owner registry</h4>
+
+<p>
+Each broker partition is consumed by a single consumer within a given consumer group. The consumer must establish its ownership of a given partition before any consumption can begin. To establish its ownership, a consumer writes its own id in an ephemeral node under the particular broker partition it is claiming.
+</p>
+
+<pre>
+/consumers/[group_id]/owners/[topic]/[broker_id-partition_id] --> consumer_node_id (ephemeral node)
+</pre>
+
+<h4>Broker node registration</h4>
+
+<p>
+The broker nodes are basically independent, so they only publish information about what they have. When a broker joins, it registers itself under the broker node registry directory and writes information about its host name and port. The broker also register the list of existing topics and their logical partitions in the broker topic registry. New topics are registered dynamically when they are created on the broker.
+</p>
+
+<h4>Consumer registration algorithm</h4>
+
+<p>
+When a consumer starts, it does the following:
+<ol>
+   <li> Register itself in the consumer id registry under its group.
+   </li>
+   <li> Register a watch on changes (new consumers joining or any existing consumers leaving) under the consumer id registry. (Each change triggers rebalancing among all consumers within the group to which the changed consumer belongs.)
+   </li>
+   <li> Register a watch on changes (new brokers joining or any existing brokers leaving) under the broker id registry. (Each change triggers rebalancing among all consumers in all consumer groups.) </li>
+   <li> If the consumer creates a message stream using a topic filter, it also registers a watch on changes (new topics being added) under the broker topic registry. (Each change will trigger re-evaluation of the available topics to determine which topics are allowed by the topic filter. A new allowed topic will trigger rebalancing among all consumers within the consumer group.)</li>
+   <li> Force itself to rebalance within in its consumer group.
+   </li>
+</ol>
+</p>
+
+<h4>Consumer rebalancing algorithm</h4>
+<p>
+The consumer rebalancing algorithms allows all the consumers in a group to come into consensus on which consumer is consuming which partitions. Consumer rebalancing is triggered on each addition or removal of both broker nodes and other consumers within the same group. For a given topic and a given consumer group, broker partitions are divided evenly among consumers within the group. A partition is always consumed by a single consumer. This design simplifies the implementation. Had we allowed a partition to be concurrently consumed by multiple consumers, there would be contention on the partition and some kind of locking would be required. If there are more consumers than partitions, some consumers won't get any data at all. During rebalancing, we try to assign partitions to consumers in such a way that reduces the number of broker nodes each consumer has to connect to.
+</p>
+<p>
+Each consumer does the following during rebalancing:
+</p>
+<pre>
+   1. For each topic T that C<sub>i</sub> subscribes to 
+   2.   let P<sub>T</sub> be all partitions producing topic T
+   3.   let C<sub>G</sub> be all consumers in the same group as C<sub>i</sub> that consume topic T
+   4.   sort P<sub>T</sub> (so partitions on the same broker are clustered together)
+   5.   sort C<sub>G</sub>
+   6.   let i be the index position of C<sub>i</sub> in C<sub>G</sub> and let N = size(P<sub>T</sub>)/size(C<sub>G</sub>)
+   7.   assign partitions from i*N to (i+1)*N - 1 to consumer C<sub>i</sub>
+   8.   remove current entries owned by C<sub>i</sub> from the partition owner registry
+   9.   add newly assigned partitions to the partition owner registry
+        (we may need to re-try this until the original partition owner releases its ownership)
+</pre>
+<p>
+When rebalancing is triggered at one consumer, rebalancing should be triggered in other consumers within the same group about the same time.
+</p>

Added: kafka/site/081/introduction.html
URL: http://svn.apache.org/viewvc/kafka/site/081/introduction.html?rev=1574659&view=auto
==============================================================================
--- kafka/site/081/introduction.html (added)
+++ kafka/site/081/introduction.html Wed Mar  5 21:06:36 2014
@@ -0,0 +1,82 @@
+<h3><a id="introduction">1.1 Introduction</a></h3>
+Kafka is a distributed, partitioned, replicated commit log service. It provides the functionality of a messaging system, but with a unique design.
+<p>
+What does all that mean?
+<p>
+First let's review some basic messaging terminology:
+<ul>
+	<li>Kafka maintains feeds of messages in categories called <i>topics</i>.
+	<li>We'll call processes that publish messages to a Kafka topic <i>producers</i>.
+	<li>We'll call processes that subscribe to topics and process the feed of published messages <i>consumers</i>..
+	<li>Kafka is run as a cluster comprised of one or more servers each of which is called a <i>broker</i>.
+</ul>
+
+So, at a high level, producers send messages over the network to the Kafka cluster which in turn serves them up to consumers like this:
+<div style="text-align: center; width: 100%">
+  <img src="../images/producer_consumer.png">
+</div>
+
+Communication between the clients and the servers is done with a simple, high-performance, language agnostic <a href="https://cwiki.apache.org/confluence/display/KAFKA/A+Guide+To+The+Kafka+Protocol">TCP protocol</a>. We provide a java client for Kafka, but clients are available in <a href="https://cwiki.apache.org/confluence/display/KAFKA/Clients">many languages</a>.
+
+<h4>Topics and Logs</h4>
+Let's first dive into the high-level abstraction Kafka provides&mdash;the topic.
+<p>
+A topic is a category or feed name to which messages are published. For each topic, the Kafka cluster maintains a partitioned log that looks like this:
+<div style="text-align: center; width: 100%">
+  <img src="../images/log_anatomy.png">
+</div>
+Each partition is an ordered, immutable sequence of messages that is continually appended to&mdash;a commit log. The messages in the partitions are each assigned a sequential id number called the <i>offset</i> that uniquely identifies each message within the partition.
+<p>
+The Kafka cluster retains all published messages&mdash;whether or not they have been consumed&mdash;for a configurable period of time. For example if the log retention is set to two days, then for the two days after a message is published it is available for consumption, after which it will be discarded to free up space. Kafka's performance is effectively constant with respect to data size so retaining lots of data is not a problem.
+<p>
+In fact the only metadata retained on a per-consumer basis is the position of the consumer in in the log, called the "offset". This offset is controlled by the consumer: normally a consumer will advance its offset linearly as it reads messages, but in fact the position is controlled by the consumer and it can consume messages in any order it likes. For example a consumer can reset to an older offset to reprocess.
+<p>
+This combination of features means that Kafka consumers are very cheap&mdash;they can come and go without much impact on the cluster or on other consumers. For example, you can use our command line tools to "tail" the contents of any topic without changing what is consumed by any existing consumers.
+<p>
+The partitions in the log serve several purposes. First, they allow the log to scale beyond a size that will fit on a single server. Each individual partition must fit on the servers that host it, but a topic may have many partitions so it can handle an arbitrary amount of data. Second they act as the unit of parallelism&mdash;more on that in a bit. 
+
+<h4>Distribution</h4>
+
+The partitions of the log are distributed over the servers in the Kafka cluster with each server handling data and requests for a share of the partitions. Each partition is replicated across a configurable number of servers for fault tolerance.
+<p>
+Each partition has one server which acts as the "leader" and zero or more servers which act as "followers". The leader handles all read and write requests for the partition while the followers passively replicate the leader. If the leader fails, one of the followers will automatically become the new leader. Each server acts as a leader for some of its partitions and a follower for others so load is well balanced within the cluster.
+
+<h4>Producers</h4>
+
+Producers publish data to the topics of their choice. The producer is able to chose which message to assign to which partition within the topic. This can be done in a round-robin fashion simply to balance load or it can be done according to some semantic partition function (say based on some key in the message). More on the use of partitioning in a second.
+
+<h4>Consumers</h4>
+
+Messaging traditionally has two models: <a href="http://en.wikipedia.org/wiki/Message_queue">queuing</a> and <a href="http://en.wikipedia.org/wiki/Publish%E2%80%93subscribe_pattern">publish-subscribe</a>. In a queue, a pool of consumers may read from a server and each message goes to one of them; in publish-subscribe the message is broadcast to all consumers. Kafka offers a single consumer abstraction that generalizes both of these&mdash;the <i>consumer group</i>.
+<p>
+Consumers label themselves with a consumer group name, and each message published to a topic is delivered to one consumer instance within each subscribing consumer group. Consumer instances can be in separate processes or on separate machines.
+<p>
+If all the consumer instances have the same consumer group, then this works just like a traditional queue balancing load over the consumers.
+<p>
+If all the consumers instances have different consumer groups then this works like publish-subscribe and all messages are broadcast to all consumers. 
+<p>
+More commonly, however, we have found that topics have a small number of consumer groups, one for each "logical subscriber". Each group is composed of many consumer instances for scalability and fault tolerance. This is nothing more than publish-subscribe semantics where the subscriber is cluster of consumers instead of a single process.
+<p>
+
+<div style="float: right; margin: 20px; width: 500px" class="caption">
+  <img src="../images/consumer-groups.png"><br>
+  A two server Kafka cluster hosting four partitions (P0-P3) with two consumer groups. Consumer group A has two consumer instances and group B has four.
+</div>
+<p>
+Kafka has stronger ordering guarantees than a traditional messaging system too.
+<p>
+A traditional queue retains messages in-order on the server, and if multiple consumers consume from the queue then the server hands out messages in the order they are stored. However although the server hands out messages in order, the messages are delivered asynchronously to consumers, so they may arrive out of order on different consumers. This effectively means the ordering of the messages is lost in the presence of parallel consumption. Messaging systems often work around this by having a notion of "exclusive consumer" that allows only on process to consume from a queue, but of course this means that there is no parallelism in processing.
+<p>
+Kafka does it better. By having a notion of parallelism&mdash;the partition&mdash;within the topics, Kafka is able to provide both ordering guarantees and load balancing over a pool of consumer processes. This is achieved by assigning the partitions in the topic to the consumers in the consumer group so that each partition is consumed by exactly one consumer in the group. By doing this we ensure that the consumer is the only reader of that partition and consumes the data in order. Since there are many partitions this still balances the load over many consumer instances. Note however that there cannot be more consumer instances than partitions.
+<p>
+Not that partitioning means Kafka only provides a total order over messages <i>within</i> a partition. This combined with the ability to partition data by key is sufficient for the vast majority of applications. However, if you require a total order over messages this can be achieved with a topic that has only one partition, though this will mean only one consumer process.
+
+<h4>Guarantees</h4>
+
+At a high-level Kafka gives the following guarantees
+<ul>
+  <li>Messages sent by a producer to a particular topic partition will be appended in the order they are sent. That is if a message M1 is sent by the same producer as a message M2, and M1 is sent first, then M1 will have a lower offset then M2 and appear earlier in the log.
+  <li>A consumer instance sees messages in the order they are stored in the log.
+  <li>For a topic with replication factor N, we will tolerate up to N-1 server failures without losing any messages committed to the log.
+</ul>
+More details on these guarantees are given in the design section of the documentation.
\ No newline at end of file