You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kafka.apache.org by ju...@apache.org on 2014/10/24 17:55:01 UTC

svn commit: r1634079 [3/3] - in /kafka/site/082: ./ api.html configuration.html design.html documentation.html ecosystem.html implementation.html introduction.html migration.html ops.html quickstart.html upgrade.html uses.html

Added: kafka/site/082/introduction.html
URL: http://svn.apache.org/viewvc/kafka/site/082/introduction.html?rev=1634079&view=auto
==============================================================================
--- kafka/site/082/introduction.html (added)
+++ kafka/site/082/introduction.html Fri Oct 24 15:55:00 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 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 responsible for choosing 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><a id="intro_consumers">Consumers</a></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 consumer 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 one 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>
+Kafka only provides a total order over messages <i>within</i> a partition, not between different partitions in a topic. Per-partition ordering combined with the ability to partition data by key is sufficient for most 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 than 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.

Added: kafka/site/082/migration.html
URL: http://svn.apache.org/viewvc/kafka/site/082/migration.html?rev=1634079&view=auto
==============================================================================
--- kafka/site/082/migration.html (added)
+++ kafka/site/082/migration.html Fri Oct 24 15:55:00 2014
@@ -0,0 +1,17 @@
+<!--#include virtual="../includes/header.html" -->
+<h2>Migrating from 0.7.x to 0.8</h2>
+
+0.8 is our first (and hopefully last) release with a non-backwards-compatible wire protocol, ZooKeeper     layout, and on-disk data format. This was a chance for us to clean up a lot of cruft and start fresh. This means performing a no-downtime upgrade is more painful than normal&mdash;you cannot just swap in the new code in-place.
+
+<h3>Migration Steps</h3>
+
+<ol>
+    <li>Setup a new cluster running 0.8.
+    <li>Use the 0.7 to 0.8 <a href="tools.html">migration tool</a> to mirror data from the 0.7 cluster into the 0.8 cluster.
+    <li>When the 0.8 cluster is fully caught up, redeploy all data <i>consumers</i> running the 0.8 client and reading from the 0.8 cluster.
+    <li>Finally migrate all 0.7 producers to 0.8 client publishing data to the 0.8 cluster.
+    <li>Decomission the 0.7 cluster.
+    <li>Drink.
+</ol>
+
+<!--#include virtual="../includes/footer.html" -->
\ No newline at end of file

Added: kafka/site/082/ops.html
URL: http://svn.apache.org/viewvc/kafka/site/082/ops.html?rev=1634079&view=auto
==============================================================================
--- kafka/site/082/ops.html (added)
+++ kafka/site/082/ops.html Fri Oct 24 15:55:00 2014
@@ -0,0 +1,602 @@
+Here is some information on actually running Kafka as a production system based on usage and experience at LinkedIn. Please send us any additional tips you know of.
+
+<h3><a id="basic_ops">6.1 Basic Kafka Operations</a></h3>
+
+This section will review the most common operations you will perform on your Kafka cluster. All of the tools reviewed in this section are available under the <code>bin/</code> directory of the Kafka distribution and each tool will print details on all possible commandline options if it is run with no arguments.
+    
+<h4><a id="basic_ops_add_topic">Adding and removing topics</a></h4>
+
+You have the option of either adding topics manually or having them be created automatically when data is first published to a non-existent topic. If topics are auto-created then you may want to tune the default <a href="#topic-config">topic configurations</a> used for auto-created topics.
+<p>
+Topics are added and modified using the topic tool:
+<pre>
+ &gt; bin/kafka-topics.sh --zookeeper zk_host:port/chroot --create --topic my_topic_name 
+       --partitions 20 --replication-factor 3 --config x=y
+</pre>
+The replication factor controls how many servers will replicate each message that is written. If you have a replication factor of 3 then up to 2 servers can fail before you will lose access to your data. We recommend you use a replication factor of 2 or 3 so that you can transparently bounce machines without interrupting data consumption.
+<p>
+The partition count controls how many logs the topic will be sharded into. There are several impacts of the partition count. First each partition must fit entirely on a single server. So if you have 20 partitions the full data set (and read and write load) will be handled by no more than 20 servers (no counting replicas). Finally the partition count impacts the maximum parallelism of your consumers. This is discussed in greater detail in the <a href="#intro_consumers">concepts section</a>.
+<p>
+The configurations added on the command line override the default settings the server has for things like the length of time data should be retained. The complete set of per-topic configurations is documented <a href="#topic-config">here</a>.
+
+<h4><a id="basic_ops_modify_topic">Modifying topics</a></h4>
+
+You can change the configuration or partitioning of a topic using the same topic tool.
+<p>
+To add partitions you can do
+<pre>
+ &gt; bin/kafka-topics.sh --zookeeper zk_host:port/chroot --alter --topic my_topic_name 
+       --partitions 40 
+</pre>
+Be aware that one use case for partitions is to semantically partition data, and adding partitions doesn't change the partitioning of existing data so this may disturb consumers if they rely on that partition. That is if data is partitioned by <code>hash(key) % number_of_partitions</code> then this partitioning will potentially be shuffled by adding partitions but Kafka will not attempt to automatically redistribute data in any way.
+<p>
+To add configs:
+<pre>
+ &gt; bin/kafka-topics.sh --zookeeper zk_host:port/chroot --alter --topic my_topic_name --config x=y
+</pre>
+To remove a config:
+<pre>
+ &gt; bin/kafka-topics.sh --zookeeper zk_host:port/chroot --alter --topic my_topic_name --deleteConfig x
+</pre>
+And finally deleting a topic:
+<pre>
+ &gt; bin/kafka-topics.sh --zookeeper zk_host:port/chroot --delete --topic my_topic_name
+</pre>
+WARNING: Delete topic functionality is beta in 0.8.1. Please report any bugs that you encounter on the <a href="mailto: users@kafka.apache.org">mailing list</a> or <a href="https://issues.apache.org/jira/browse/KAFKA">JIRA</a>.
+<p>
+Kafka does not currently support reducing the number of partitions for a topic or changing the replication factor.
+
+<h4><a id="basic_ops_restarting">Graceful shutdown</a></h4>
+
+The Kafka cluster will automatically detect any broker shutdown or failure and elect new leaders for the partitions on that machine. This will occur whether a server fails or it is brought down intentionally for maintenance or configuration changes. For the later cases Kafka supports a more graceful mechanism for stoping a server then just killing it.
+
+When a server is stopped gracefully it has two optimizations it will take advantage of:
+<ol>
+    <li>It will sync all its logs to disk to avoid needing to do any log recovery when it restarts (i.e. validating the checksum for all messages in the tail of the log). Log recovery takes time so this speeds up intentional restarts.
+    <li>It will migrate any partitions the server is the leader for to other replicas prior to shutting down. This will make the leadership transfer faster and minimize the time each partition is unavailable to a few milliseconds.
+</ol>
+
+Syncing the logs will happen automatically happen whenever the server is stopped other than by a hard kill, but the controlled leadership migration requires using a special setting:
+<pre>
+    controlled.shutdown.enable=true
+</pre>
+Note that controlled shutdown will only succeed if <i>all</i> the partitions hosted on the broker have replicas (i.e. the replication factor is greater than 1 <i>and</i> at least one of these replicas is alive). This is generally what you want since shutting down the last replica would make that topic partition unavailable.
+
+<h4><a id="basic_ops_leader_balancing">Balancing leadership</a></h4>
+
+Whenever a broker stops or crashes leadership for that broker's partitions transfers to other replicas. This means that by default when the broker is restarted it will only be a follower for all its partitions, meaning it will not be used for client reads and writes.
+<p>
+To avoid this imbalance, Kafka has a notion of preferred replicas. If the list of replicas for a partition is 1,5,9 then node 1 is preferred as the leader to either node 5 or 9 because it is earlier in the replica list. You can have the Kafka cluster try to restore leadership to the restored replicas by running the command:
+<pre>
+ &gt; bin/kafka-preferred-replica-election.sh --zookeeper zk_host:port/chroot
+</pre>
+
+Since running this command can be tedious you can also configure Kafka to do this automatically by setting the following configuration:
+<pre>
+    auto.leader.rebalance.enable=true
+</pre>
+
+<h4><a id="basic_ops_mirror_maker">Mirroring data between clusters</a></h4>
+
+We refer to the process of replicating data <i>between</i> Kafka clusters "mirroring" to avoid confusion with the replication that happens amongst the nodes in a single cluster. Kafka comes with a tool for mirroring data between Kafka clusters. The tool reads from one or more source clusters and writes to a destination cluster, like this:
+<p>
+<img src="/images/mirror-maker.png">
+<p>
+A common use case for this kind of mirroring is to provide a replica in another datacenter. This scenario will be discussed in more detail in the next section.
+<p>
+You can run many such mirroring processes to increase throughput and for fault-tolerance (if one process dies, the others will take overs the additional load).
+<p>
+Data will be read from topics in the source cluster and written to a topic with the same name in the destination cluster. In fact the mirror maker is little more than a Kafka consumer and producer hooked together.
+<p>
+The source and destination clusters are completely independent entities: they can have different numbers of partitions and the offsets will not be the same. For this reason the mirror cluster is not really intended as a fault-tolerance mechanism (as the consumer position will be different); for that we recommend using normal in-cluster replication. The mirror maker process will, however, retain and use the message key for partitioning so order is preserved on a per-key basis.
+<p>
+Here is an example showing how to mirror a single topic (named <i>my-topic</i>) from two input clusters:
+<pre>
+ &gt; bin/kafka-run-class.sh kafka.tools.MirrorMaker
+       --consumer.config consumer-1.properties --consumer.config consumer-2.properties 
+       --producer.config producer.properties --whitelist my-topic
+</pre>
+Note that we specify the list of topics with the <code>--whitelist</code> option. This option allows any regular expression using <a href="http://docs.oracle.com/javase/7/docs/api/java/util/regex/Pattern.html">Java-style regular expressions</a>. So you could mirror two topics named <i>A</i> and <i>B</i> using <code>--whitelist 'A|B'</code>. Or you could mirror <i>all</i> topics using <code>--whitelist '*'</code>. Make sure to quote any regular expression to ensure the shell doesn't try to expand it as a file path. For convenience we allow the use of ',' instead of '|' to specify a list of topics.
+<p>
+Sometime it is easier to say what it is that you <i>don't</i> want. Instead of using <code>--whitelist</code> to say what you want to mirror you can use <code>--blacklist</code> to say what to exclude. This also takes a regular expression argument.
+<p>
+Combining mirroring with the configuration <code>auto.create.topics.enable=true</code> makes it possible to have a replica cluster that will automatically create and replicate all data in a source cluster even as new topics are added.
+
+<h4><a id="basic_ops_consumer_lag">Checking consumer position</a></h4>
+Sometimes it's useful to see the position of your consumers. We have a tool that will show the position of all consumers in a consumer group as well as how far behind the end of the log they are. To run this tool on a consumer group named <i>my-group</i> consuming a topic named <i>my-topic</i> would look like this:
+<pre>
+ &gt; bin/kafka-run-class.sh kafka.tools.ConsumerOffsetChecker --zkconnect localhost:2181 --group test
+Group           Topic                          Pid Offset          logSize         Lag             Owner
+my-group        my-topic                       0   0               0               0               test_jkreps-mn-1394154511599-60744496-0
+my-group        my-topic                       1   0               0               0               test_jkreps-mn-1394154521217-1a0be913-0
+</pre>
+
+<h4><a id="basic_ops_cluster_expansion">Expanding your cluster</a></h4>
+
+Adding servers to a Kafka cluster is easy, just assign them a unique broker id and start up Kafka on your new servers. However these new servers will not automatically be assigned any data partitions, so unless partitions are moved to them they won't be doing any work until new topics are created. So usually when you add machines to your cluster you will want to migrate some existing data to these machines.
+<p>
+The process of migrating data is manually initiated but fully automated. Under the covers what happens is that Kafka will add the new server as a follower of the partition it is migrating and allow it to fully replicate the existing data in that partition. When the new server has fully replicated the contents of this partition and joined the in-sync replica one of the existing replicas will delete their partition's data.
+<p>
+The partition reassignment tool can be used to move partitions across brokers. An ideal partition distribution would ensure even data load and partition sizes across all brokers. In 0.8.1, the partition reassignment tool does not have the capability to automatically study the data distribution in a Kafka cluster and move partitions around to attain an even load distribution. As such, the admin has to figure out which topics or partitions should be moved around. 
+<p>
+The partition reassignment tool can run in 3 mutually exclusive modes -
+<ul>
+<li>--generate: In this mode, given a list of topics and a list of brokers, the tool generates a candidate reassignment to move all partitions of the specified topics to the new brokers. This option merely provides a convenient way to generate a partition reassignment plan given a list of topics and target brokers.</li>
+<li>--execute: In this mode, the tool kicks off the reassignment of partitions based on the user provided reassignment plan. (using the --reassignment-json-file option). This can either be a custom reassignment plan hand crafted by the admin or provided by using the --generate option</li>
+<li>--verify: In this mode, the tool verifies the status of the reassignment for all partitions listed during the last --execute. The status can be either of successfully completed, failed or in progress</li>
+</ul>
+<h5>Automatically migrating data to new machines</h5>
+The partition reassignment tool can be used to move some topics off of the current set of brokers to the newly added brokers. This is typically useful while expanding an existing cluster since it is easier to move entire topics to the new set of brokers, than moving one partition at a time. When used to do this, the user should provide a list of topics that should be moved to the new set of brokers and a target list of new brokers. The tool then evenly distributes all partitions for the given list of topics across the new set of brokers. During this move, the replication factor of the topic is kept constant. Effectively the replicas for all partitions for the input list of topics are moved from the old set of brokers to the newly added brokers. 
+<p>
+For instance, the following example will move all partitions for topics foo1,foo2 to the new set of brokers 5,6. At the end of this move, all partitions for topics foo1 and foo2 will <i>only</i> exist on brokers 5,6
+<p>
+Since, the tool accepts the input list of topics as a json file, you first need to identify the topics you want to move and create the json file as follows-
+<pre>
+> cat topics-to-move.json
+{"topics": [{"topic": "foo1"},
+            {"topic": "foo2"}],
+ "version":1
+}
+</pre>
+Once the json file is ready, use the partition reassignment tool to generate a candidate assignment-
+<pre>
+> bin/kafka-reassign-partitions.sh --zookeeper localhost:2181 --topics-to-move-json-file topics-to-move.json --broker-list "5,6" --generate 
+Current partition replica assignment
+
+{"version":1,
+ "partitions":[{"topic":"foo1","partition":2,"replicas":[1,2]},
+               {"topic":"foo1","partition":0,"replicas":[3,4]},
+               {"topic":"foo2","partition":2,"replicas":[1,2]},
+               {"topic":"foo2","partition":0,"replicas":[3,4]},
+               {"topic":"foo1","partition":1,"replicas":[2,3]},
+               {"topic":"foo2","partition":1,"replicas":[2,3]}]
+}
+
+Proposed partition reassignment configuration
+
+{"version":1,
+ "partitions":[{"topic":"foo1","partition":2,"replicas":[5,6]},
+               {"topic":"foo1","partition":0,"replicas":[5,6]},
+               {"topic":"foo2","partition":2,"replicas":[5,6]},
+               {"topic":"foo2","partition":0,"replicas":[5,6]},
+               {"topic":"foo1","partition":1,"replicas":[5,6]},
+               {"topic":"foo2","partition":1,"replicas":[5,6]}]
+}
+</pre>
+<p>
+The tool generates a candidate assignment that will move all partitions from topics foo1,foo2 to brokers 5,6. Note, however, that at this point, the partition movement has not started, it merely tells you the current assignment and the proposed new assignment. The current assignment should be saved in case you want to rollback to it. The new assignment should be saved in a json file (e.g. expand-cluster-reassignment.json) to be input to the tool with the --execute option as follows-
+<pre>
+> bin/kafka-reassign-partitions.sh --zookeeper localhost:2181 --reassignment-json-file expand-cluster-reassignment.json --execute
+Current partition replica assignment
+
+{"version":1,
+ "partitions":[{"topic":"foo1","partition":2,"replicas":[1,2]},
+               {"topic":"foo1","partition":0,"replicas":[3,4]},
+               {"topic":"foo2","partition":2,"replicas":[1,2]},
+               {"topic":"foo2","partition":0,"replicas":[3,4]},
+               {"topic":"foo1","partition":1,"replicas":[2,3]},
+               {"topic":"foo2","partition":1,"replicas":[2,3]}]
+}
+
+Save this to use as the --reassignment-json-file option during rollback
+Successfully started reassignment of partitions
+{"version":1,
+ "partitions":[{"topic":"foo1","partition":2,"replicas":[5,6]},
+               {"topic":"foo1","partition":0,"replicas":[5,6]},
+               {"topic":"foo2","partition":2,"replicas":[5,6]},
+               {"topic":"foo2","partition":0,"replicas":[5,6]},
+               {"topic":"foo1","partition":1,"replicas":[5,6]},
+               {"topic":"foo2","partition":1,"replicas":[5,6]}]
+}
+</pre>
+<p>
+Finally, the --verify option can be used with the tool to check the status of the partition reassignment. Note that the same expand-cluster-reassignment.json (used with the --execute option) should be used with the --verify option
+<pre>
+> bin/kafka-reassign-partitions.sh --zookeeper localhost:2181 --reassignment-json-file expand-cluster-reassignment.json --verify
+Status of partition reassignment:
+Reassignment of partition [foo1,0] completed successfully
+Reassignment of partition [foo1,1] is in progress
+Reassignment of partition [foo1,2] is in progress
+Reassignment of partition [foo2,0] completed successfully
+Reassignment of partition [foo2,1] completed successfully 
+Reassignment of partition [foo2,2] completed successfully 
+</pre>
+
+<h5>Custom partition assignment and migration</h5>
+The partition reassignment tool can also be used to selectively move replicas of a partition to a specific set of brokers. When used in this manner, it is assumed that the user knows the reassignment plan and does not require the tool to generate a candidate reassignment, effectively skipping the --generate step and moving straight to the --execute step
+<p>
+For instance, the following example moves partition 0 of topic foo1 to brokers 5,6 and partition 1 of topic foo2 to brokers 2,3
+<p>
+The first step is to hand craft the custom reassignment plan in a json file-
+<pre>
+> cat custom-reassignment.json
+{"version":1,"partitions":[{"topic":"foo1","partition":0,"replicas":[5,6]},{"topic":"foo2","partition":1,"replicas":[2,3]}]}
+</pre>
+Then, use the json file with the --execute option to start the reassignment process-
+<pre>
+> bin/kafka-reassign-partitions.sh --zookeeper localhost:2181 --reassignment-json-file custom-reassignment.json --execute
+Current partition replica assignment
+
+{"version":1,
+ "partitions":[{"topic":"foo1","partition":0,"replicas":[1,2]},
+               {"topic":"foo2","partition":1,"replicas":[3,4]}]
+}
+
+Save this to use as the --reassignment-json-file option during rollback
+Successfully started reassignment of partitions
+{"version":1,
+ "partitions":[{"topic":"foo1","partition":0,"replicas":[5,6]},
+               {"topic":"foo2","partition":1,"replicas":[2,3]}]
+}
+</pre>
+<p>
+The --verify option can be used with the tool to check the status of the partition reassignment. Note that the same expand-cluster-reassignment.json (used with the --execute option) should be used with the --verify option
+<pre>
+bin/kafka-reassign-partitions.sh --zookeeper localhost:2181 --reassignment-json-file custom-reassignment.json --verify
+Status of partition reassignment:
+Reassignment of partition [foo1,0] completed successfully
+Reassignment of partition [foo2,1] completed successfully 
+</pre>
+
+<h4><a id="basic_ops_decommissioning_brokers">Decommissioning brokers</a></h4>
+The partition reassignment tool does not have the ability to automatically generate a reassignment plan for decommissioning brokers yet. As such, the admin has to come up with a reassignment plan to move the replica for all partitions hosted on the broker to be decommissioned, to the rest of the brokers. This can be relatively tedious as the reassignment needs to ensure that all the replicas are not moved from the decommissioned broker to only one other broker. To make this process effortless, we plan to add tooling support for decommissioning brokers in 0.8.2.
+
+<h4><a id="basic_ops_increase_replication_factor">Increasing replication factor</a></h4>
+Increasing the replication factor of an existing partition is easy. Just specify the extra replicas in the custom reassignment json file and use it with the --execute option to increase the replication factor of the specified partitions. 
+<p>
+For instance, the following example increases the replication factor of partition 0 of topic foo from 1 to 3. Before increasing the replication factor, the partition's only replica existed on broker 5. As part of increasing the replication factor, we will add more replicas on brokers 6 and 7.
+<p>
+The first step is to hand craft the custom reassignment plan in a json file-
+<pre>
+> cat increase-replication-factor.json
+{"version":1,
+ "partitions":[{"topic":"foo","partition":0,"replicas":[5,6,7]}]}
+</pre>
+Then, use the json file with the --execute option to start the reassignment process-
+<pre>
+> bin/kafka-reassign-partitions.sh --zookeeper localhost:2181 --reassignment-json-file increase-replication-factor.json --execute
+Current partition replica assignment
+
+{"version":1,
+ "partitions":[{"topic":"foo","partition":0,"replicas":[5]}]}
+
+Save this to use as the --reassignment-json-file option during rollback
+Successfully started reassignment of partitions
+{"version":1,
+ "partitions":[{"topic":"foo","partition":0,"replicas":[5,6,7]}]}
+</pre>
+<p>
+The --verify option can be used with the tool to check the status of the partition reassignment. Note that the same increase-replication-factor.json (used with the --execute option) should be used with the --verify option
+<pre>
+bin/kafka-reassign-partitions.sh --zookeeper localhost:2181 --reassignment-json-file increase-replication-factor.json --verify
+Status of partition reassignment:
+Reassignment of partition [foo,0] completed successfully
+</pre>
+You can also verify the increase in replication factor with the kafka-topics tool-
+<pre>
+> bin/kafka-topics.sh --zookeeper localhost:2181 --topic foo --describe
+Topic:foo	PartitionCount:1	ReplicationFactor:3	Configs:
+	Topic: foo	Partition: 0	Leader: 5	Replicas: 5,6,7	Isr: 5,6,7
+</pre>
+
+<h3><a id="datacenters">6.2 Datacenters</a></h3>
+
+Some deployments will need to manage a data pipeline that spans multiple datacenters. Our recommended approach to this is to deploy a local Kafka cluster in each datacenter with application instances in each datacenter interacting only with their local cluster and mirroring between clusters (see the documentation on the <a href="#basic_ops_mirror_maker">mirror maker tool</a> for how to do this).
+<p>
+This deployment pattern allows datacenters to act as independent entities and allows us to manage and tune inter-datacenter replication centrally. This allows each facility to stand alone and operate even if the inter-datacenter links are unavailable: when this occurs the mirroring falls behind until the link is restored at which time it catches up.
+<p>
+For applications that need a global view of all data you can use mirroring to provide clusters which have aggregate data mirrored from the local clusters in <i>all</i> datacenters. These aggregate clusters are used for reads by applications that require the full data set.
+<p>
+This is not the only possible deployment pattern. It is possible to read from or write to a remote Kafka cluster over the WAN, though obviously this will add whatever latency is required to get the cluster.
+<p>
+Kafka naturally batches data in both the producer and consumer so it can achieve high-throughput even over a high-latency connection. To allow this though it may be necessary to increase the TCP socket buffer sizes for the producer, consumer, and broker using the <code>socket.send.buffer.bytes</code> and <code>socket.receive.buffer.bytes</code> configurations. The appropriate way to set this is documented <a href="http://en.wikipedia.org/wiki/Bandwidth-delay_product">here</a>.    
+<p>
+It is generally <i>not</i> advisable to run a <i>single</i> Kafka cluster that spans multiple datacenters over a high-latency link. This will incur very high replication latency both for Kafka writes and ZooKeeper writes, and neither Kafka nor ZooKeeper will remain available in all locations if the network between locations is unavailable.
+
+<h3><a id="config">6.3 Kafka Configuration</a></h3>
+
+<h4><a id="clientconfig">Important Client Configurations</a></h4>
+The most important producer configurations control
+<ul>
+    <li>compression</li>
+    <li>sync vs async production</li>
+    <li>batch size (for async producers)</li>
+</ul>
+The most important consumer configuration is the fetch size.
+<p>
+All configurations are documented in the <a href="#configuration">configuration</a> section.
+<p>
+<h4><a id="prodconfig">A Production Server Config</a></h4>
+Here is our server production server configuration:
+<pre>
+# Replication configurations
+num.replica.fetchers=4
+replica.fetch.max.bytes=1048576
+replica.fetch.wait.max.ms=500
+replica.high.watermark.checkpoint.interval.ms=5000
+replica.socket.timeout.ms=30000
+replica.socket.receive.buffer.bytes=65536
+replica.lag.time.max.ms=10000
+replica.lag.max.messages=4000
+
+controller.socket.timeout.ms=30000
+controller.message.queue.size=10
+
+# Log configuration
+num.partitions=8
+message.max.bytes=1000000
+auto.create.topics.enable=true
+log.index.interval.bytes=4096
+log.index.size.max.bytes=10485760
+log.retention.hours=168
+log.flush.interval.ms=10000
+log.flush.interval.messages=20000
+log.flush.scheduler.interval.ms=2000
+log.roll.hours=168
+log.retention.check.interval.ms=300000
+log.segment.bytes=1073741824
+
+# ZK configuration
+zookeeper.connection.timeout.ms=6000
+zookeeper.sync.time.ms=2000
+
+# Socket server configuration
+num.io.threads=8
+num.network.threads=8
+socket.request.max.bytes=104857600
+socket.receive.buffer.bytes=1048576
+socket.send.buffer.bytes=1048576
+queued.max.requests=16
+fetch.purgatory.purge.interval.requests=100
+producer.purgatory.purge.interval.requests=100
+</pre>
+
+Our client configuration varies a fair amount between different use cases.
+
+<h3><a id="java">Java Version</a></h3>
+We're currently running JDK 1.7 u51, and we've switched over to the G1 collector. If you do this (and we highly recommend it), make sure you're on u51. We tried out u21 in testing, but we had a number of problems with the GC implementation in that version.
+
+Our tuning looks like this:
+<pre>
+-Xms4g -Xmx4g -XX:PermSize=48m -XX:MaxPermSize=48m -XX:+UseG1GC
+-XX:MaxGCPauseMillis=20 -XX:InitiatingHeapOccupancyPercent=35
+</pre>
+
+For reference, here are the stats on one of LinkedIn's busiest clusters (at peak):
+        - 15 brokers
+        - 15.5k partitions (replication factor 2)
+        - 400k messages/sec in
+        - 70 MB/sec inbound, 400 MB/sec+ outbound
+
+The tuning looks fairly aggressive, but all of the brokers in that cluster have a 90% GC pause time of about 21ms, and they're doing less than 1 young GC per second.
+	
+<h3><a id="hwandos">6.4 Hardware and OS</a></h3>
+We are using dual quad-core Intel Xeon machines with 24GB of memory.
+<p>
+You need sufficient memory to buffer active readers and writers. You can do a back-of-the-envelope estimate of memory needs by assuming you want to be able to buffer for 30 seconds and compute your memory need as write_throughput*30.
+<p>
+The disk throughput is important. We have 8x7200 rpm SATA drives. In general disk throughput is the performance bottleneck, and more disks is more better. Depending on how you configure flush behavior you may or may not benefit from more expensive disks (if you force flush often then higher RPM SAS drives may be better).
+
+<h4><a id="os">OS</a></h4>
+Kafka should run well on any unix system and has been tested on Linux and Solaris.
+<p>
+We have seen a few issues running on Windows and Windows is not currently a well supported platform though we would be happy to change that.
+<p>
+You likely don't need to do much OS-level tuning though there are a few things that will help performance. 
+<p>
+Two configurations that may be important:
+<ul>
+    <li>We upped the number of file descriptors since we have lots of topics and lots of connections.
+    <li>We upped the max socket buffer size to enable high-performance data transfer between data centers <a href="http://www.psc.edu/index.php/networking/641-tcp-tune">described here</a>.
+</ul>
+
+<h4><a id="diskandfs">Disks and Filesystem</a></h4>
+We recommend using multiple drives to get good throughput and not sharing the same drives used for Kafka data with application logs or other OS filesystem activity to ensure good latency. As of 0.8 you can either RAID these drives together into a single volume or format and mount each drive as its own directory. Since Kafka has replication the redundancy provided by RAID can also be provided at the application level. This choice has several tradeoffs.
+<p>
+If you configure multiple data directories partitions will be assigned round-robin to data directories. Each partition will be entirely in one of the data directories. If data is not well balanced among partitions this can lead to load imbalance between disks.
+<p>
+RAID can potentially do better at balancing load between disks (although it doesn't always seem to) because it balances load at a lower level. The primary downside of RAID is that it is usually a big performance hit for write throughput and reduces the available disk space.
+<p>
+Another potential benefit of RAID is the ability to tolerate disk failures. However our experience has been that rebuilding the RAID array is so I/O intensive that it effectively disables the server, so this does not provide much real availability improvement.
+
+<h4><a id="appvsosflush">Application vs. OS Flush Management</a></h4>
+Kafka always immediately writes all data to the filesystem and supports the ability to configure the flush policy that controls when data is forced out of the OS cache and onto disk using the and flush. This flush policy can be controlled to force data to disk after a period of time or after a certain number of messages has been written. There are several choices in this configuration.
+<p>
+Kafka must eventually call fsync to know that data was flushed. When recovering from a crash for any log segment not known to be fsync'd Kafka will check the integrity of each message by checking its CRC and also rebuild the accompanying offset index file as part of the recovery process executed on startup.
+<p>
+Note that durability in Kafka does not require syncing data to disk, as a failed node will always recover from its replicas.
+<p>
+We recommend using the default flush settings which disable application fsync entirely. This means relying on the background flush done by the OS and Kafka's own background flush. This provides the best of all worlds for most uses: no knobs to tune, great throughput and latency, and full recovery guarantees. We generally feel that the guarantees provided by replication are stronger than sync to local disk, however the paranoid still may prefer having both and application level fsync policies are still supported.
+<p>
+The drawback of using application level flush settings are that this is less efficient in it's disk usage pattern (it gives the OS less leeway to re-order writes) and it can introduce latency as fsync in most Linux filesystems blocks writes to the file whereas the background flushing does much more granular page-level locking.
+<p>
+In general you don't need to do any low-level tuning of the filesystem, but in the next few sections we will go over some of this in case it is useful.
+
+<h4><a id="linuxflush">Understanding Linux OS Flush Behavior</a></h4>
+
+In Linux, data written to the filesystem is maintained in <a href="http://en.wikipedia.org/wiki/Page_cache">pagecache</a> until it must be written out to disk (due to an application-level fsync or the OS's own flush policy). The flushing of data is done by a set of background threads called pdflush (or in post 2.6.32 kernels "flusher threads").
+<p>
+Pdflush has a configurable policy that controls how much dirty data can be maintained in cache and for how long before it must be written back to disk. This policy is described <a href="http://www.westnet.com/~gsmith/content/linux-pdflush.htm">here</a>. When Pdflush cannot keep up with the rate of data being written it will eventually cause the writing process to block incurring latency in the writes to slow down the accumulation of data.
+<p>
+You can see the current state of OS memory usage by doing
+<pre>
+  &gt; cat /proc/meminfo
+</pre>
+The meaning of these values are described in the link above.
+<p>
+Using pagecache has several advantages over an in-process cache for storing data that will be written out to disk:
+<ul>
+  <li>The I/O scheduler will batch together consecutive small writes into bigger physical writes which improves throughput.
+  <li>The I/O scheduler will attempt to re-sequence writes to minimize movement of the disk head which improves throughput.
+  <li>It automatically uses all the free memory on the machine
+</ul>
+
+<h4><a id="ext4">Ext4 Notes</a></h4>
+Ext4 may or may not be the best filesystem for Kafka. Filesystems like XFS supposedly handle locking during fsync better. We have only tried Ext4, though.
+<p>
+It is not necessary to tune these settings, however those wanting to optimize performance have a few knobs that will help:
+<ul>
+  <li>data=writeback: Ext4 defaults to data=ordered which puts a strong order on some writes. Kafka does not require this ordering as it does very paranoid data recovery on all unflushed log. This setting removes the ordering constraint and seems to significantly reduce latency.
+  <li>Disabling journaling: Journaling is a tradeoff: it makes reboots faster after server crashes but it introduces a great deal of additional locking which adds variance to write performance. Those who don't care about reboot time and want to reduce a major source of write latency spikes can turn off journaling entirely.
+  <li>commit=num_secs: This tunes the frequency with which ext4 commits to its metadata journal. Setting this to a lower value reduces the loss of unflushed data during a crash. Setting this to a higher value will improve throughput.
+  <li>nobh: This setting controls additional ordering guarantees when using data=writeback mode. This should be safe with Kafka as we do not depend on write ordering and improves throughput and latency.
+  <li>delalloc: Delayed allocation means that the filesystem avoid allocating any blocks until the physical write occurs. This allows ext4 to allocate a large extent instead of smaller pages and helps ensure the data is written sequentially. This feature is great for throughput. It does seem to involve some locking in the filesystem which adds a bit of latency variance.
+</ul>
+	
+<h3><a id="monitoring">6.6 Monitoring</a></h3>
+
+Kafka uses Yammer Metrics for metrics reporting in both the server and the client. This can be configured to report stats using pluggable stats reporters to hook up to your monitoring system.
+<p>
+The easiest way to see the available metrics to fire up jconsole and point it at a running kafka client or server; this will all browsing all metrics with JMX.
+<p>
+We pay particular we do graphing and alerting on the following metrics:
+<table class="data-table">
+<tbody><tr>
+      <th>Description</th>
+      <th>Mbean name</th>
+      <th>Normal value</th>
+    </tr>
+    <tr>
+      <td>Message in rate</td>
+      <td>"kafka.server":name="AllTopicsMessagesInPerSec",type="BrokerTopicMetrics"</td>
+      <td></td>
+    </tr>
+    <tr>
+      <td>Byte in rate</td>
+      <td>"kafka.server":name="AllTopicsBytesInPerSec",type="BrokerTopicMetrics"</td>
+      <td></td>
+    </tr>
+    <tr>
+      <td>Request rate</td>
+      <td>"kafka.network":name="{Produce|Fetch-consumer|Fetch-follower}-RequestsPerSec",type="RequestMetrics"</td>
+      <td></td>
+    </tr>
+    <tr>
+      <td>Byte out rate</td>
+      <td>"kafka.server":name="AllTopicsBytesOutPerSec",type="BrokerTopicMetrics"</td>
+      <td></td>
+    </tr>
+    <tr>
+      <td>Log flush rate and time</td>
+      <td>"kafka.log":name="LogFlushRateAndTimeMs",type="LogFlushStats"</td>
+      <td></td>
+    </tr>
+    <tr>
+      <td># of under replicated partitions (|ISR| &lt |all replicas|)</td>
+      <td>"kafka.server":name="UnderReplicatedPartitions",type="ReplicaManager"</td>
+      <td>0</td>
+    </tr>
+    <tr>
+      <td>Is controller active on broker</td>
+      <td>"kafka.controller":name="ActiveControllerCount",type="KafkaController"</td>
+      <td>only one broker in the cluster should have 1</td>
+    </tr>
+    <tr>
+      <td>Leader election rate</td>
+      <td>"kafka.controller":name="LeaderElectionRateAndTimeMs",type="ControllerStats"</td>
+      <td>non-zero when there are broker failures</td>
+    </tr>
+    <tr>
+      <td>Unclean leader election rate</td>
+      <td>"kafka.controller":name="UncleanLeaderElectionsPerSec",type="ControllerStats"</td>
+      <td>0</td>
+    </tr>
+    <tr>
+      <td>Partition counts</td>
+      <td>"kafka.server":name="PartitionCount",type="ReplicaManager"</td>
+      <td>mostly even across brokers</td>
+    </tr>
+    <tr>
+      <td>Leader replica counts</td>
+      <td>"kafka.server":name="LeaderCount",type="ReplicaManager"</td>
+      <td>mostly even across brokers</td>
+    </tr>
+    <tr>
+      <td>ISR shrink rate</td>
+      <td>"kafka.server":name="ISRShrinksPerSec",type="ReplicaManager"</td>
+      <td>If a broker goes down, ISR for some of the partitions will
+	shrink. When that broker is up again, ISR will be expanded
+	once the replicas are fully caught up. Other than that, the
+	expected value for both ISR shrink rate and expansion rate is 0. </td>
+    </tr>
+    <tr>
+      <td>ISR expansion rate</td>
+      <td>"kafka.server":name="ISRExpandsPerSec",type="ReplicaManager"</td>
+      <td>See above</td>
+    </tr>
+    <tr>
+      <td>Max lag in messages btw follower and leader replicas</td>
+      <td>"kafka.server":name="([-.\w]+)-MaxLag",type="ReplicaFetcherManager"</td>
+      <td>&lt replica.lag.max.messages</td>
+    </tr>
+    <tr>
+      <td>Lag in messages per follower replica</td>
+      <td>"kafka.server":name="([-.\w]+)-ConsumerLag",type="FetcherLagMetrics"</td>
+      <td>&lt replica.lag.max.messages</td>
+    </tr>
+    <tr>
+      <td>Requests waiting in the producer purgatory</td>
+      <td>"kafka.server":name="PurgatorySize",type="ProducerRequestPurgatory"</td>
+      <td>non-zero if ack=-1 is used</td>
+    </tr>
+    <tr>
+      <td>Requests waiting in the fetch purgatory</td>
+      <td>"kafka.server":name="PurgatorySize",type="FetchRequestPurgatory"</td>
+      <td>size depends on fetch.wait.max.ms in the consumer</td>
+    </tr>
+    <tr>
+      <td>Request total time</td>
+      <td>"kafka.network":name="{Produce|Fetch-Consumer|Fetch-Follower}-TotalTimeMs",type="RequestMetrics"</td>
+      <td>broken into queue, local, remote and response send time</td>
+    </tr>
+    <tr>
+      <td>Time the request waiting in the request queue</td>
+      <td>"kafka.network":name="{Produce|Fetch-Consumer|Fetch-Follower}-QueueTimeMs",type="RequestMetrics"</td>
+      <td></td>
+    </tr>
+    <tr>
+      <td>Time the request being processed at the leader</td>
+      <td>"kafka.network":name="{Produce|Fetch-Consumer|Fetch-Follower}-LocalTimeMs",type="RequestMetrics"</td>
+      <td></td>
+    </tr>
+    <tr>
+      <td>Time the request waits for the follower</td>
+      <td>"kafka.network":name="{Produce|Fetch-Consumer|Fetch-Follower}-RemoteTimeMs",type="RequestMetrics"</td>
+      <td>non-zero for produce requests when ack=-1</td>
+    </tr>
+    <tr>
+      <td>Time to send the response</td>
+      <td>"kafka.network":name="{Produce|Fetch-Consumer|Fetch-Follower}-ResponseSendTimeMs",type="RequestMetrics"</td>
+      <td></td>
+    </tr>
+    <tr>
+      <td>Number of messages the consumer lags behind the producer by</td>
+      <td>"kafka.consumer":name="([-.\w]+)-MaxLag",type="ConsumerFetcherManager"</td>
+      <td></td>
+    </tr>
+</tbody></table>
+
+We recommend monitor GC time and other stats and various server stats such as CPU utilization, I/O service time, etc.
+
+On the client side, we recommend monitor the message/byte rate (global and per topic), request rate/size/time, and on the consumer side, max lag in messages among all partitions and min fetch request rate. For a consumer to keep up, max lag needs to be less than a threshold and min fetch rate needs to be larger than 0.
+
+<h4>Audit</h4>
+The final alerting we do is on the correctness of the data delivery. We audit that every message that is sent is consumed by all consumers and measure the lag for this to occur. For important topics we alert if a certain completeness is not achieved in a certain time period. The details of this are discussed in KAFKA-260.
+
+<h3><a id="zk">6.7 ZooKeeper</a></h3>
+
+<h4><a id="zkversion">Stable version</a></h4>
+At LinkedIn, we are running ZooKeeper 3.3.*. Version 3.3.3 has known serious issues regarding ephemeral node deletion and session expirations. After running into those issues in production, we upgraded to 3.3.4 and have been running that smoothly for over a year now.
+
+<h4><a id="zkops">Operationalizing ZooKeeper</a></h4>
+Operationally, we do the following for a healthy ZooKeeper installation:
+<ul>
+  <li>Redundancy in the physical/hardware/network layout: try not to put them all in the same rack, decent (but don't go nuts) hardware, try to keep redundant power and network paths, etc.</li>
+  <li>I/O segregation: if you do a lot of write type traffic you'll almost definitely want the transaction logs on a different disk group than application logs and snapshots (the write to the ZooKeeper service has a synchronous write to disk, which can be slow).</li>
+  <li>Application segregation: Unless you really understand the application patterns of other apps that you want to install on the same box, it can be a good idea to run ZooKeeper in isolation (though this can be a balancing act with the capabilities of the hardware).</li>
+  <li>Use care with virtualization: It can work, depending on your cluster layout and read/write patterns and SLAs, but the tiny overheads introduced by the virtualization layer can add up and throw off ZooKeeper, as it can be very time sensitive</li>
+  <li>ZooKeeper configuration and monitoring: It's java, make sure you give it 'enough' heap space (We usually run them with 3-5G, but that's mostly due to the data set size we have here). Unfortunately we don't have a good formula for it. As far as monitoring, both JMZ and the 4 letter commands are very useful, they do overlap in some cases (and in those cases we prefer the 4 letter commands, they seem more predictable, or at the very least, they work better with the LI monitoring infrastructure)</li>
+  <li>Don't overbuild the cluster: large clusters, especially in a write heavy usage pattern, means a lot of intracluster communication (quorums on the writes and subsequent cluster member updates), but don't underbuild it (and risk swamping the cluster).</li>
+  <li>Try to run on a 3-5 node cluster: ZooKeeper writes use quorums and inherently that means having an odd number of machines in a cluster. Remember that a 5 node cluster will cause writes to slow down compared to a 3 node cluster, but will allow more fault tolerance.</li>
+</ul>
+Overall, we try to keep the ZooKeeper system as small as will handle the load (plus standard growth capacity planning) and as simple as possible. We try not to do anything fancy with the configuration or application layout as compared to the official release as well as keep it as self contained as possible. For these reasons, we tend to skip the OS packaged versions, since it has a tendency to try to put things in the OS standard hierarchy, which can be 'messy', for want of a better way to word it.

Added: kafka/site/082/quickstart.html
URL: http://svn.apache.org/viewvc/kafka/site/082/quickstart.html?rev=1634079&view=auto
==============================================================================
--- kafka/site/082/quickstart.html (added)
+++ kafka/site/082/quickstart.html Fri Oct 24 15:55:00 2014
@@ -0,0 +1,172 @@
+<h3><a id="quickstart">1.3 Quick Start</a></h3>
+
+This tutorial assumes you are starting fresh and have no existing Kafka or ZooKeeper data.
+
+<h4> Step 1: Download the code </h4>
+
+<a href="https://www.apache.org/dyn/closer.cgi?path=/kafka/0.8.1.1/kafka_2.9.2-0.8.1.1.tgz" title="Kafka downloads">Download</a> the 0.8.1.1 release and un-tar it.
+
+<pre>
+&gt; <b>tar -xzf kafka_2.9.2-0.8.1.1.tgz</b>
+&gt; <b>cd kafka_2.9.2-0.8.1.1</b>
+</pre>
+
+<h4>Step 2: Start the server</h4>
+
+<p>
+Kafka uses ZooKeeper so you need to first start a ZooKeeper server if you don't already have one. You can use the convenience script packaged with kafka to get a quick-and-dirty single-node ZooKeeper instance.
+
+<pre>
+&gt; <b>bin/zookeeper-server-start.sh config/zookeeper.properties</b>
+[2013-04-22 15:01:37,495] INFO Reading configuration from: config/zookeeper.properties (org.apache.zookeeper.server.quorum.QuorumPeerConfig)
+...
+</pre>
+
+Now start the Kafka server:
+<pre>
+&gt; <b>bin/kafka-server-start.sh config/server.properties</b>
+[2013-04-22 15:01:47,028] INFO Verifying properties (kafka.utils.VerifiableProperties)
+[2013-04-22 15:01:47,051] INFO Property socket.send.buffer.bytes is overridden to 1048576 (kafka.utils.VerifiableProperties)
+...
+</pre>
+
+<h4>Step 3: Create a topic</h4>
+
+Let's create a topic named "test" with a single partition and only one replica:
+<pre>
+&gt; <b>bin/kafka-topics.sh --create --zookeeper localhost:2181 --replication-factor 1 --partitions 1 --topic test</b>
+</pre>
+
+We can now see that topic if we run the list topic command:
+<pre>
+&gt; <b>bin/kafka-topics.sh --list --zookeeper localhost:2181</b>
+test
+</pre>
+Alternatively, instead of manually creating topics you can also configure your brokers to auto-create topics when a non-existent topic is published to.
+
+<h4>Step 4: Send some messages</h4>
+
+Kafka comes with a command line client that will take input from a file or from standard input and send it out as messages to the Kafka cluster. By default each line will be sent as a separate message.
+<p>
+Run the producer and then type a few messages into the console to send to the server.
+
+<pre>
+&gt; <b>bin/kafka-console-producer.sh --broker-list localhost:9092 --topic test</b> 
+<b>This is a message</b>
+<b>This is another message</b>
+</pre>
+
+<h4>Step 5: Start a consumer</h4>
+
+Kafka also has a command line consumer that will dump out messages to standard output.
+
+<pre>
+&gt; <b>bin/kafka-console-consumer.sh --zookeeper localhost:2181 --topic test --from-beginning</b>
+This is a message
+This is another message
+</pre>
+<p>
+If you have each of the above commands running in a different terminal then you should now be able to type messages into the producer terminal and see them appear in the consumer terminal.
+</p>
+<p>
+All of the command line tools have additional options; running the command with no arguments will display usage information documenting them in more detail.	
+</p>
+
+<h4>Step 6: Setting up a multi-broker cluster</h4>
+
+So far we have been running against a single broker, but that's no fun. For Kafka, a single broker is just a cluster of size one, so nothing much changes other than starting a few more broker instances. But just to get feel for it, let's expand our cluster to three nodes (still all on our local machine).
+<p>
+First we make a config file for each of the brokers:
+<pre>
+&gt; <b>cp config/server.properties config/server-1.properties</b> 
+&gt; <b>cp config/server.properties config/server-2.properties</b>
+</pre>
+
+Now edit these new files and set the following properties:
+<pre>
+ 
+config/server-1.properties:
+    broker.id=1
+    port=9093
+    log.dir=/tmp/kafka-logs-1
+ 
+config/server-2.properties:
+    broker.id=2
+    port=9094
+    log.dir=/tmp/kafka-logs-2
+</pre>
+The <code>broker.id</code> property is the unique and permanent name of each node in the cluster. We have to override the port and log directory only because we are running these all on the same machine and we want to keep the brokers from all trying to register on the same port or overwrite each others data.
+<p>
+We already have Zookeeper and our single node started, so we just need to start the two new nodes:
+<pre>
+&gt; <b>bin/kafka-server-start.sh config/server-1.properties &amp;</b>
+...
+&gt; <b>bin/kafka-server-start.sh config/server-2.properties &amp;</b>
+...
+</pre>
+
+Now create a new topic with a replication factor of three:
+<pre>
+&gt; <b>bin/kafka-topics.sh --create --zookeeper localhost:2181 --replication-factor 3 --partitions 1 --topic my-replicated-topic</b>
+</pre>
+
+Okay but now that we have a cluster how can we know which broker is doing what? To see that run the "describe topics" command:
+<pre>
+&gt; <b>bin/kafka-topics.sh --describe --zookeeper localhost:2181 --topic my-replicated-topic</b>
+Topic:my-replicated-topic	PartitionCount:1	ReplicationFactor:3	Configs:
+	Topic: my-replicated-topic	Partition: 0	Leader: 1	Replicas: 1,2,0	Isr: 1,2,0
+</pre>
+Here is an explanation of output. The first line gives a summary of all the partitions, each additional line gives information about one partition. Since we have only one partition for this topic there is only one line.
+<ul>
+  <li>"leader" is the node responsible for all reads and writes for the given partition. Each node will be the leader for a randomly selected portion of the partitions.
+  <li>"replicas" is the list of nodes that replicate the log for this partition regardless of whether they are the leader or even if they are currently alive.
+  <li>"isr" is the set of "in-sync" replicas. This is the subset of the replicas list that is currently alive and caught-up to the leader.
+</ul> 
+Note that in my example node 1 is the leader for the only partition of the topic.
+<p>
+We can run the same command on the original topic we created to see where it is:
+<pre>
+&gt; <b>bin/kafka-topics.sh --describe --zookeeper localhost:2181 --topic test</b>
+Topic:test	PartitionCount:1	ReplicationFactor:1	Configs:
+	Topic: test	Partition: 0	Leader: 0	Replicas: 0	Isr: 0
+</pre>
+So there is no surprise there&mdash;the original topic has no replicas and is on server 0, the only server in our cluster when we created it.
+<p>
+Let's publish a few messages to our new topic:
+<pre>
+&gt; <b>bin/kafka-console-producer.sh --broker-list localhost:9092 --topic my-replicated-topic</b>
+...
+<b>my test message 1</b>
+<b>my test message 2</b>
+<b>^C</b> 
+</pre>
+Now let's consume these messages:
+<pre>
+&gt; <b>bin/kafka-console-consumer.sh --zookeeper localhost:2181 --from-beginning --topic my-replicated-topic</b>
+...
+my test message 1
+my test message 2
+<b>^C</b>
+</pre>
+
+Now let's test out fault-tolerance. Broker 1 was acting as the leader so let's kill it:
+<pre>
+&gt; <b>ps | grep server-1.properties</b>
+<i>7564</i> ttys002    0:15.91 /System/Library/Frameworks/JavaVM.framework/Versions/1.6/Home/bin/java...
+&gt; <b>kill -9 7564</b>
+</pre>
+
+Leadership has switched to one of the slaves and node 1 is no longer in the in-sync replica set:
+<pre>
+&gt; <b>bin/kafka-topics.sh --describe --zookeeper localhost:2181 --topic my-replicated-topic</b>
+Topic:my-replicated-topic	PartitionCount:1	ReplicationFactor:3	Configs:
+	Topic: my-replicated-topic	Partition: 0	Leader: 2	Replicas: 1,2,0	Isr: 2,0
+</pre>
+But the messages are still be available for consumption even though the leader that took the writes originally is down:
+<pre>
+&gt; <b>bin/kafka-console-consumer.sh --zookeeper localhost:2181 --from-beginning --topic my-replicated-topic</b>
+...
+my test message 1
+my test message 2
+<b>^C</b>
+</pre>

Added: kafka/site/082/upgrade.html
URL: http://svn.apache.org/viewvc/kafka/site/082/upgrade.html?rev=1634079&view=auto
==============================================================================
--- kafka/site/082/upgrade.html (added)
+++ kafka/site/082/upgrade.html Fri Oct 24 15:55:00 2014
@@ -0,0 +1,8 @@
+<h3><a id="upgrade">1.5 Upgrading From Previous Versions</a></h3>
+<h4>Upgrading from 0.8.0 to 0.8.1</h4>
+
+0.8.1 is fully compatible with 0.8. The upgrade can be done one broker at a time by simply bringing it down, updating the code, and restarting it.
+
+<h4>Upgrading from 0.7</h4>
+
+0.8, the release in which added replication, was our first backwards-incompatible release: major changes were made to the API, ZooKeeper data structures, and protocol, and configuration. The upgrade from 0.7 to 0.8.x requires a <a href="https://cwiki.apache.org/confluence/display/KAFKA/Migrating+from+0.7+to+0.8">special tool</a> for migration. This migration can be done without downtime.

Added: kafka/site/082/uses.html
URL: http://svn.apache.org/viewvc/kafka/site/082/uses.html?rev=1634079&view=auto
==============================================================================
--- kafka/site/082/uses.html (added)
+++ kafka/site/082/uses.html Fri Oct 24 15:55:00 2014
@@ -0,0 +1,39 @@
+<h3><a id="uses">1.2 Use Cases</a></h3>
+
+Here is a description of a few of the popular use cases for Apache Kafka. For an overview of a number of these areas in action, see <a href="http://engineering.linkedin.com/distributed-systems/log-what-every-software-engineer-should-know-about-real-time-datas-unifying">this blog post</a>.
+
+<h4>Messaging</h4>
+
+Kafka works well as a replacement for a more traditional message broker. Message brokers are used for a variety of reasons (to decouple processing from data producers, to buffer unprocessed messages, etc). In comparison to most messaging systems Kafka has better throughput, built-in partitioning, replication, and fault-tolerance which makes it a good solution for large scale message processing applications.
+<p>
+In our experience messaging uses are often comparatively low-throughput, but may require low end-to-end latency and often depend on the strong durability guarantees Kafka provides.
+<p>
+In this domain Kafka is comparable to traditional messaging systems such as <a href="http://activemq.apache.org">ActiveMQ</a> or <a href="https://www.rabbitmq.com">RabbitMQ</a>.
+
+<h4>Website Activity Tracking</h4>
+
+The original use case for Kafka was to be able to rebuild a user activity tracking pipeline as a set of real-time publish-subscribe feeds. This means site activity (page views, searches, or other actions users may take) is published to central topics with one topic per activity type. These feeds are available for subscription for a range of use cases including real-time processing, real-time monitoring, and loading into Hadoop or offline data warehousing systems for offline processing and reporting.
+<p>
+Activity tracking is often very high volume as many activity messages are generated for each user page view.
+
+<h4>Metrics</h4>
+
+Kafka is often used for operational monitoring data. This involves aggregating statistics from distributed applications to produce centralized feeds of operational data.
+
+<h4>Log Aggregation</h4>
+
+Many people use Kafka as a replacement for a log aggregation solution. Log aggregation typically collects physical log files off servers and puts them in a central place (a file server or HDFS perhaps) for processing. Kafka abstracts away the details of files and gives a cleaner abstraction of log or event data as a stream of messages. This allows for lower-latency processing and easier support for multiple data sources and distributed data consumption.
+
+In comparison to log-centric systems like Scribe or Flume, Kafka offers equally good performance, stronger durability guarantees due to replication, and much lower end-to-end latency.
+
+<h4>Stream Processing</h4>
+
+Many users end up doing stage-wise processing of data where data is consumed from topics of raw data and then aggregated, enriched, or otherwise transformed into new Kafka topics for further consumption. For example a processing flow for article recommendation might crawl article content from RSS feeds and publish it to an "articles" topic; further processing might help normalize or deduplicate this content to a topic of cleaned article content; a final stage might attempt to match this content to users. This creates a graph of real-time data flow out of the individual topics. <a href="https://github.com/nathanmarz/storm">Storm</a> and <a href="http://samza.incubator.apache.org/">Samza</a> are popular frameworks for implementing these kinds of transformations.
+
+<h4>Event Sourcing</h4>
+
+<a href="http://martinfowler.com/eaaDev/EventSourcing.html">Event sourcing</a> is a style of application design where state changes are logged as a time-ordered sequence of records. Kafka's support for very large stored log data makes it an excellent backend for an application built in this style.
+
+<h4>Commit Log</h4>
+
+Kafka can serve as a kind of external commit-log for a distributed system. The log helps replicate data between nodes and acts as a re-syncing mechanism for failed nodes to restore their data. The <a href="/documentation.html#compaction">log compaction</a> feature in Kafka helps support this usage. In this usage Kafka is similar to <a href="http://zookeeper.apache.org/bookkeeper/">Apache BookKeeper</a> project.
\ No newline at end of file