kafka-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From damian...@apache.org
Subject [2/2] kafka-site git commit: MINOR: add Interactive Queries docs
Date Fri, 07 Jul 2017 13:21:56 GMT
MINOR: add Interactive Queries docs


Project: http://git-wip-us.apache.org/repos/asf/kafka-site/repo
Commit: http://git-wip-us.apache.org/repos/asf/kafka-site/commit/422095a8
Tree: http://git-wip-us.apache.org/repos/asf/kafka-site/tree/422095a8
Diff: http://git-wip-us.apache.org/repos/asf/kafka-site/diff/422095a8

Branch: refs/heads/asf-site
Commit: 422095a8f8eac42406c343e2ff45d7825b1b4b0f
Parents: a6523f9
Author: Damian Guy <damian.guy@gmail.com>
Authored: Fri Jul 7 14:21:46 2017 +0100
Committer: Damian Guy <damian.guy@gmail.com>
Committed: Fri Jul 7 14:21:46 2017 +0100

----------------------------------------------------------------------
 0110/api.html                                   |   8 +-
 0110/documentation/index.html                   |  17 +
 0110/generated/producer_config.html             |   2 +-
 0110/generated/topic_config.html                |   4 +-
 0110/images/streams-interactive-queries-01.png  | Bin 0 -> 80976 bytes
 0110/images/streams-interactive-queries-02.png  | Bin 0 -> 73218 bytes
 0110/images/streams-interactive-queries-03.png  | Bin 0 -> 79879 bytes
 .../streams-interactive-queries-api-01.png      | Bin 0 -> 84438 bytes
 .../streams-interactive-queries-api-02.png      | Bin 0 -> 100725 bytes
 0110/implementation.html                        | 125 ++--
 0110/js/templateData.js                         |   5 +-
 0110/ops.html                                   |   4 +-
 0110/quickstart.html                            |   8 +-
 0110/streams/architecture.html                  |  98 +--
 0110/streams/core-concepts.html                 |   6 +-
 0110/streams/developer-guide.html               | 746 ++++++++++++++++---
 0110/streams/index.html                         |  15 +-
 0110/streams/upgrade-guide.html                 |  36 +-
 18 files changed, 821 insertions(+), 253 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kafka-site/blob/422095a8/0110/api.html
----------------------------------------------------------------------
diff --git a/0110/api.html b/0110/api.html
index 9777186..05768a1 100644
--- a/0110/api.html
+++ b/0110/api.html
@@ -39,7 +39,7 @@
 		&lt;dependency&gt;
 			&lt;groupId&gt;org.apache.kafka&lt;/groupId&gt;
 			&lt;artifactId&gt;kafka-clients&lt;/artifactId&gt;
-			&lt;version&gt;{{fullDotVersion}}&lt;/version&gt;
+			&lt;version&gt;{{dotVersion}}.0&lt;/version&gt;
 		&lt;/dependency&gt;
 	</pre>
 
@@ -55,7 +55,7 @@
 		&lt;dependency&gt;
 			&lt;groupId&gt;org.apache.kafka&lt;/groupId&gt;
 			&lt;artifactId&gt;kafka-clients&lt;/artifactId&gt;
-			&lt;version&gt;{{fullDotVersion}}&lt;/version&gt;
+			&lt;version&gt;{{dotVersion}}.0&lt;/version&gt;
 		&lt;/dependency&gt;
 	</pre>
 
@@ -74,7 +74,7 @@
 		&lt;dependency&gt;
 			&lt;groupId&gt;org.apache.kafka&lt;/groupId&gt;
 			&lt;artifactId&gt;kafka-streams&lt;/artifactId&gt;
-			&lt;version&gt;{{fullDotVersion}}&lt;/version&gt;
+			&lt;version&gt;{{dotVersion}}.0&lt;/version&gt;
 		&lt;/dependency&gt;
 	</pre>
 
@@ -96,7 +96,7 @@
 		&lt;dependency&gt;
 			&lt;groupId&gt;org.apache.kafka&lt;/groupId&gt;
 			&lt;artifactId&gt;kafka-clients&lt;/artifactId&gt;
-			&lt;version&gt;{{fullDotVersion}}&lt;/version&gt;
+			&lt;version&gt;{{dotVersion}}.0&lt;/version&gt;
 		&lt;/dependency&gt;
 	</pre>
 	For more information about the AdminClient APIs, see the <a href="/{{version}}/javadoc/index.html?org/apache/kafka/clients/admin/AdminClient.html"
title="Kafka {{dotVersion}} Javadoc">javadoc</a>.

http://git-wip-us.apache.org/repos/asf/kafka-site/blob/422095a8/0110/documentation/index.html
----------------------------------------------------------------------
diff --git a/0110/documentation/index.html b/0110/documentation/index.html
index 698eeed..1d7507f 100644
--- a/0110/documentation/index.html
+++ b/0110/documentation/index.html
@@ -1 +1,18 @@
+<!--
+ Licensed to the Apache Software Foundation (ASF) under one or more
+ contributor license agreements.  See the NOTICE file distributed with
+ this work for additional information regarding copyright ownership.
+ The ASF licenses this file to You under the Apache License, Version 2.0
+ (the "License"); you may not use this file except in compliance with
+ the License.  You may obtain a copy of the License at
+
+    http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+-->
+
 <!--#include virtual="../documentation.html" -->
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/kafka-site/blob/422095a8/0110/generated/producer_config.html
----------------------------------------------------------------------
diff --git a/0110/generated/producer_config.html b/0110/generated/producer_config.html
index 819dfbc..2fd1891 100644
--- a/0110/generated/producer_config.html
+++ b/0110/generated/producer_config.html
@@ -112,5 +112,5 @@
 <tr>
 <td>transaction.timeout.ms</td><td>The maximum amount of time in ms that
the transaction coordinator will wait for a transaction status update from the producer before
proactively aborting the ongoing transaction.If this value is larger than the max.transaction.timeout.ms
setting in the broker, the request will fail with a `InvalidTransactionTimeout` error.</td><td>int</td><td>60000</td><td></td><td>low</td></tr>
 <tr>
-<td>transactional.id</td><td>The TransactionalId to use for transactional
delivery. This enables reliability semantics which span multiple producer sessions since it
allows the client to guarantee that transactions using the same TransactionalId have been
completed prior to starting any new transactions. If no TransactionalId is provided, then
the producer is limited to idempotent delivery. Note that enable.idempotence must be enabled
if a TransactionalId is configured. The default is empty, which means transactions cannot
be used.</td><td>string</td><td>null</td><td>org.apache.kafka.common.config.ConfigDef$NonEmptyString@4fca772d</td><td>low</td></tr>
+<td>transactional.id</td><td>The TransactionalId to use for transactional
delivery. This enables reliability semantics which span multiple producer sessions since it
allows the client to guarantee that transactions using the same TransactionalId have been
completed prior to starting any new transactions. If no TransactionalId is provided, then
the producer is limited to idempotent delivery. Note that enable.idempotence must be enabled
if a TransactionalId is configured. The default is empty, which means transactions cannot
be used.</td><td>string</td><td>null</td><td>non-empty
string</td><td>low</td></tr>
 </tbody></table>

http://git-wip-us.apache.org/repos/asf/kafka-site/blob/422095a8/0110/generated/topic_config.html
----------------------------------------------------------------------
diff --git a/0110/generated/topic_config.html b/0110/generated/topic_config.html
index 38587fc..d5e8835 100644
--- a/0110/generated/topic_config.html
+++ b/0110/generated/topic_config.html
@@ -21,11 +21,11 @@
 <tr>
 <td>flush.ms</td><td>This setting allows specifying a time interval at
which we will force an fsync of data written to the log. For example if this was set to 1000
we would fsync after 1000 ms had passed. In general we recommend you not set this and use
replication for durability and allow the operating system's background flush capabilities
as it is more efficient.</td><td>long</td><td>9223372036854775807</td><td>[0,...]</td><td>log.flush.interval.ms</td><td>medium</td></tr>
 <tr>
-<td>follower.replication.throttled.replicas</td><td>A list of replicas
for which log replication should be throttled on the follower side. The list should describe
a set of replicas in the form [PartitionId]:[BrokerId],[PartitionId]:[BrokerId]:... or alternatively
the wildcard '*' can be used to throttle all replicas for this topic.</td><td>list</td><td>""</td><td>kafka.server.ThrottledReplicaListValidator$@52feb982</td><td>follower.replication.throttled.replicas</td><td>medium</td></tr>
+<td>follower.replication.throttled.replicas</td><td>A list of replicas
for which log replication should be throttled on the follower side. The list should describe
a set of replicas in the form [PartitionId]:[BrokerId],[PartitionId]:[BrokerId]:... or alternatively
the wildcard '*' can be used to throttle all replicas for this topic.</td><td>list</td><td>""</td><td>[partitionId],[brokerId]:[partitionId],[brokerId]:...</td><td>follower.replication.throttled.replicas</td><td>medium</td></tr>
 <tr>
 <td>index.interval.bytes</td><td>This setting controls how frequently Kafka
adds an index entry to it's offset index. The default setting ensures that we index a message
roughly every 4096 bytes. More indexing allows reads to jump closer to the exact position
in the log but makes the index larger. You probably don't need to change this.</td><td>int</td><td>4096</td><td>[0,...]</td><td>log.index.interval.bytes</td><td>medium</td></tr>
 <tr>
-<td>leader.replication.throttled.replicas</td><td>A list of replicas for
which log replication should be throttled on the leader side. The list should describe a set
of replicas in the form [PartitionId]:[BrokerId],[PartitionId]:[BrokerId]:... or alternatively
the wildcard '*' can be used to throttle all replicas for this topic.</td><td>list</td><td>""</td><td>kafka.server.ThrottledReplicaListValidator$@52feb982</td><td>leader.replication.throttled.replicas</td><td>medium</td></tr>
+<td>leader.replication.throttled.replicas</td><td>A list of replicas for
which log replication should be throttled on the leader side. The list should describe a set
of replicas in the form [PartitionId]:[BrokerId],[PartitionId]:[BrokerId]:... or alternatively
the wildcard '*' can be used to throttle all replicas for this topic.</td><td>list</td><td>""</td><td>[partitionId],[brokerId]:[partitionId],[brokerId]:...</td><td>leader.replication.throttled.replicas</td><td>medium</td></tr>
 <tr>
 <td>max.message.bytes</td><td><p>The largest record batch size allowed
by Kafka. If this is increased and there are consumers older than 0.10.2, the consumers' fetch
size must also be increased so that the they can fetch record batches this large.</p><p>In
the latest message format version, records are always grouped into batches for efficiency.
In previous message format versions, uncompressed records are not grouped into batches and
this limit only applies to a single record in that case.</p></td><td>int</td><td>1000012</td><td>[0,...]</td><td>message.max.bytes</td><td>medium</td></tr>
 <tr>

http://git-wip-us.apache.org/repos/asf/kafka-site/blob/422095a8/0110/images/streams-interactive-queries-01.png
----------------------------------------------------------------------
diff --git a/0110/images/streams-interactive-queries-01.png b/0110/images/streams-interactive-queries-01.png
new file mode 100644
index 0000000..d5d5031
Binary files /dev/null and b/0110/images/streams-interactive-queries-01.png differ

http://git-wip-us.apache.org/repos/asf/kafka-site/blob/422095a8/0110/images/streams-interactive-queries-02.png
----------------------------------------------------------------------
diff --git a/0110/images/streams-interactive-queries-02.png b/0110/images/streams-interactive-queries-02.png
new file mode 100644
index 0000000..ea894b6
Binary files /dev/null and b/0110/images/streams-interactive-queries-02.png differ

http://git-wip-us.apache.org/repos/asf/kafka-site/blob/422095a8/0110/images/streams-interactive-queries-03.png
----------------------------------------------------------------------
diff --git a/0110/images/streams-interactive-queries-03.png b/0110/images/streams-interactive-queries-03.png
new file mode 100644
index 0000000..403e3ae
Binary files /dev/null and b/0110/images/streams-interactive-queries-03.png differ

http://git-wip-us.apache.org/repos/asf/kafka-site/blob/422095a8/0110/images/streams-interactive-queries-api-01.png
----------------------------------------------------------------------
diff --git a/0110/images/streams-interactive-queries-api-01.png b/0110/images/streams-interactive-queries-api-01.png
new file mode 100644
index 0000000..2b4aaed
Binary files /dev/null and b/0110/images/streams-interactive-queries-api-01.png differ

http://git-wip-us.apache.org/repos/asf/kafka-site/blob/422095a8/0110/images/streams-interactive-queries-api-02.png
----------------------------------------------------------------------
diff --git a/0110/images/streams-interactive-queries-api-02.png b/0110/images/streams-interactive-queries-api-02.png
new file mode 100644
index 0000000..e5e7527
Binary files /dev/null and b/0110/images/streams-interactive-queries-api-02.png differ

http://git-wip-us.apache.org/repos/asf/kafka-site/blob/422095a8/0110/implementation.html
----------------------------------------------------------------------
diff --git a/0110/implementation.html b/0110/implementation.html
index 2cf401a..af234ea 100644
--- a/0110/implementation.html
+++ b/0110/implementation.html
@@ -22,61 +22,94 @@
     </p>
     <h3><a id="messages" href="#messages">5.2 Messages</a></h3>
     <p>
-    Messages consist of a fixed-size header, a variable length opaque key byte array and
a variable length opaque value byte array. The header contains the following fields:
-    <ul>
-        <li> A CRC32 checksum to detect corruption or truncation. </li>
-        <li> A format version. </li>
-        <li> An attributes identifier </li>
-        <li> A timestamp </li>
-    </ul>
-    Leaving the key and value 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>.
+    Messages consist of a variable-length header, a variable length opaque key byte array
and a variable length opaque value byte array. The format of the header is described in the
following section.
+    Leaving the key and value 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>RecordBatch</code>
interface is simply an iterator over messages with specialized methods for bulk reading and
writing to an NIO <code>Channel</code>.</p>
 
     <h3><a id="messageformat" href="#messageformat">5.3 Message Format</a></h3>
+    <p>
+    Messages (aka Records) are always written in batches. The technical term for a batch
of messages is a record batch, and a record batch contains one or more records. In the degenerate
case, we could have a record batch containing a single record.
+    Record batches and records have their own headers. The format of each is described below
for Kafka version 0.11.0 and later (message format version v2, or magic=2). <a href="https://cwiki.apache.org/confluence/display/KAFKA/A+Guide+To+The+Kafka+Protocol#AGuideToTheKafkaProtocol-Messagesets">Click
here</a> for details about message formats 0 and 1.</p>
+
+    <h4><a id="recordbatch" href="#recordbatch">5.3.1 Record Batch</a></h4>
+	<p> The following is the on-disk format of a RecordBatch. </p>
+	<p><pre class="brush: java;">
+		baseOffset: int64
+		batchLength: int32
+		partitionLeaderEpoch: int32
+		magic: int8 (current magic value is 2)
+		crc: int32
+		attributes: int16
+			bit 0~2:
+				0: no compression
+				1: gzip
+				2: snappy
+				3: lz4
+			bit 3: timestampType
+			bit 4: isTransactional (0 means not transactional)
+			bit 5: isControlBatch (0 means not a control batch)
+			bit 6~15: unused
+		lastOffsetDelta: int32
+		firstTimestamp: int64
+		maxTimestamp: int64
+		producerId: int64
+		producerEpoch: int16
+		baseSequence: int32
+		records: [Record]
+	</pre></p>
+    <p> Note that when compression is enabled, the compressed record data is serialized
directly following the count of the number of records. </p>
+
+    <p>The CRC covers the data from the attributes to the end of the batch (i.e. all
the bytes that follow the CRC). It is located after the magic byte, which
+    means that clients must parse the magic byte before deciding how to interpret the bytes
between the batch length and the magic byte. The partition leader
+    epoch field is not included in the CRC computation to avoid the need to recompute the
CRC when this field is assigned for every batch that is received by
+    the broker. The CRC-32C (Castagnoli) polynomial is used for the computation.</p>
+
+    <p>On compaction: unlike the older message formats, magic v2 and above preserves
the first and last offset/sequence numbers from the original batch when the log is cleaned.
This is required in order to be able to restore the
+    producer's state when the log is reloaded. If we did not retain the last sequence number,
for example, then after a partition leader failure, the producer might see an OutOfSequence
error. The base sequence number must
+    be preserved for duplicate checking (the broker checks incoming Produce requests for
duplicates by verifying that the first and last sequence numbers of the incoming batch match
the last from that producer). As a result,
+    it is possible to have empty batches in the log when all the records in the batch are
cleaned but batch is still retained in order to preserve a producer's last sequence number.
One oddity here is that the baseTimestamp
+    field is not preserved during compaction, so it will change if the first record in the
batch is compacted away.</p>
+
+    <h5><a id="controlbatch" href="#controlbatch">5.3.1.1 Control Batches</a></h5>
+    <p>A control batch contains a single record called the control record. Control
records should not be passed on to applications. Instead, they are used by consumers to filter
out aborted transactional messages.</p>
+    <p> The key of a control record conforms to the following schema: </p>
+    <p><pre class="brush: java">
+       version: int16 (current version is 0)
+       type: int16 (0 indicates an abort marker, 1 indicates a commit)
+    </pre></p>
+    <p>The schema for the value of a control record is dependent on the type. The value
is opaque to clients.</p>
+
+	<h4><a id="record" href="#record">5.3.2 Record</a></h4>
+	<p>Record level headers were introduced in Kafka 0.11.0. The on-disk format of a record
with Headers is delineated below. </p>
+	<p><pre class="brush: java;">
+		length: varint
+		attributes: int8
+			bit 0~7: unused
+		timestampDelta: varint
+		offsetDelta: varint
+		keyLength: varint
+		key: byte[]
+		valueLen: varint
+		value: byte[]
+		Headers => [Header]
+	</pre></p>
+	<h5><a id="recordheader" href="#recordheader">5.4.2.1 Record Header</a></h5>
+	<p><pre class="brush: java;">
+		headerKeyLength: varint
+		headerKey: String
+		headerValueLength: varint
+		Value: byte[]
+	</pre></p>
+    <p>We use the the same varint encoding as Protobuf. More information on the latter
can be found <a href="https://developers.google.com/protocol-buffers/docs/encoding#varints">here</a>.
The count of headers in a record
+    is also encoded as a varint.</p>
 
-    <pre class="brush: java;">
-       /**
-        * 1. 4 byte CRC32 of the message
-        * 2. 1 byte "magic" identifier to allow format changes, value is 0 or 1
-        * 3. 1 byte "attributes" identifier to allow annotations on the message independent
of the version
-        *    bit 0 ~ 2 : Compression codec.
-        *      0 : no compression
-        *      1 : gzip
-        *      2 : snappy
-        *      3 : lz4
-        *    bit 3 : Timestamp type
-        *      0 : create time
-        *      1 : log append time
-        *    bit 4 ~ 7 : reserved
-        * 4. (Optional) 8 byte timestamp only if "magic" identifier is greater than 0
-        * 5. 4 byte key length, containing length K
-        * 6. K byte key
-        * 7. 4 byte payload length, containing length V
-        * 8. V byte payload
-        */
-    </pre>
-    </p>
     <h3><a id="log" href="#log">5.4 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 transferred between producer, broker, and client without recopying
or conversion when desirable. This format is as follows:
+    The exact binary format for records is versioned and maintained as a standard interface
so record batches can be transferred between producer, broker, and client without recopying
or conversion when desirable. The previous section included details about the on-disk format
of records.</p>
     </p>
-    <pre class="brush: java;">
-    On-disk format of a message
-
-    offset         : 8 bytes 
-    message length : 4 bytes (value: 4 + 1 + 1 + 8(if magic value > 0) + 4 + K + 4 + V)
-    crc            : 4 bytes
-    magic value    : 1 byte
-    attributes     : 1 byte
-    timestamp      : 8 bytes (Only exists when magic value is greater than zero)
-    key length     : 4 bytes
-    key            : K bytes
-    value length   : 4 bytes
-    value          : V bytes
-    </pre>
-    <p>
+   <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
  offset is hidden from the consumer API this decision is ultimately an implementation detail
and we went with the more efficient approach.
     </p>
     <img class="centered" src="/{{version}}/images/kafka_log.png">

http://git-wip-us.apache.org/repos/asf/kafka-site/blob/422095a8/0110/js/templateData.js
----------------------------------------------------------------------
diff --git a/0110/js/templateData.js b/0110/js/templateData.js
index 3eca71e..2f32444 100644
--- a/0110/js/templateData.js
+++ b/0110/js/templateData.js
@@ -18,6 +18,5 @@ limitations under the License.
 // Define variables for doc templates
 var context={
     "version": "0110",
-    "dotVersion": "0.11.0",
-    "fullDotVersion": "0.11.0.0"
-};
+    "dotVersion": "0.11.0"
+};
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/kafka-site/blob/422095a8/0110/ops.html
----------------------------------------------------------------------
diff --git a/0110/ops.html b/0110/ops.html
index ca0cfb1..85a6e10 100644
--- a/0110/ops.html
+++ b/0110/ops.html
@@ -52,11 +52,11 @@
   <p>
   To add configs:
   <pre class="brush: bash;">
-  &gt; bin/kafka-topics.sh --zookeeper zk_host:port/chroot --alter --topic my_topic_name
--config x=y
+  &gt; bin/kafka-configs.sh --zookeeper zk_host:port/chroot --entity-type topics --entity-name
my_topic_name --alter --add-config x=y
   </pre>
   To remove a config:
   <pre class="brush: bash;">
-  &gt; bin/kafka-topics.sh --zookeeper zk_host:port/chroot --alter --topic my_topic_name
--delete-config x
+  &gt; bin/kafka-configs.sh --zookeeper zk_host:port/chroot --entity-type topics --entity-name
my_topic_name --alter --delete-config x
   </pre>
   And finally deleting a topic:
   <pre class="brush: bash;">

http://git-wip-us.apache.org/repos/asf/kafka-site/blob/422095a8/0110/quickstart.html
----------------------------------------------------------------------
diff --git a/0110/quickstart.html b/0110/quickstart.html
index 40aa2fb..6509f75 100644
--- a/0110/quickstart.html
+++ b/0110/quickstart.html
@@ -25,11 +25,11 @@ Since Kafka console scripts are different for Unix-based and Windows platforms,
 
 <h4><a id="quickstart_download" href="#quickstart_download">Step 1: Download
the code</a></h4>
 
-<a href="https://www.apache.org/dyn/closer.cgi?path=/kafka/{{fullDotVersion}}/kafka_2.11-{{fullDotVersion}}.tgz"
title="Kafka downloads">Download</a> the {{fullDotVersion}} release and un-tar it.
+<a href="https://www.apache.org/dyn/closer.cgi?path=/kafka/0.10.2.0/kafka_2.11-0.10.2.0.tgz"
title="Kafka downloads">Download</a> the 0.10.2.0 release and un-tar it.
 
 <pre class="brush: bash;">
-&gt; tar -xzf kafka_2.11-{{fullDotVersion}}.tgz
-&gt; cd kafka_2.11-{{fullDotVersion}}
+&gt; tar -xzf kafka_2.11-0.10.2.0.tgz
+&gt; cd kafka_2.11-0.10.2.0
 </pre>
 
 <h4><a id="quickstart_startserver" href="#quickstart_startserver">Step 2: Start
the server</a></h4>
@@ -187,7 +187,7 @@ my test message 2
 On Windows use:
 <pre class="brush: bash;">
 &gt; wmic process get processid,caption,commandline | find "java.exe" | find "server-1.properties"
-java.exe    java  -Xmx1G -Xms1G -server -XX:+UseG1GC ... build\libs\kafka_2.11-{{fullDotVersion}}.jar"
 kafka.Kafka config\server-1.properties    644
+java.exe    java  -Xmx1G -Xms1G -server -XX:+UseG1GC ... build\libs\kafka_2.11-0.10.2.0.jar"
 kafka.Kafka config\server-1.properties    644
 &gt; taskkill /pid 644 /f
 </pre>
 

http://git-wip-us.apache.org/repos/asf/kafka-site/blob/422095a8/0110/streams/architecture.html
----------------------------------------------------------------------
diff --git a/0110/streams/architecture.html b/0110/streams/architecture.html
index 6269994..f0def6a 100644
--- a/0110/streams/architecture.html
+++ b/0110/streams/architecture.html
@@ -20,21 +20,21 @@
 <script id="content-template" type="text/x-handlebars-template">
     <h1>Architecture</h1>
 
-    <p>Kafka Streams simplifies application development by building on the Kafka producer
and consumer libraries and leveraging the native capabilities of
-    Kafka to offer data parallelism, distributed coordination, fault tolerance, and operational
simplicity. In this section, we describe how Kafka Streams works underneath the covers.</p>
-    
+    Kafka Streams simplifies application development by building on the Kafka producer and
consumer libraries and leveraging the native capabilities of
+    Kafka to offer data parallelism, distributed coordination, fault tolerance, and operational
simplicity. In this section, we describe how Kafka Streams works underneath the covers.
+
     <p>
-    The picture below shows the anatomy of an application that uses the Kafka Streams library.
Let's walk through some details.
+        The picture below shows the anatomy of an application that uses the Kafka Streams
library. Let's walk through some details.
     </p>
     <img class="centered" src="/{{version}}/images/streams-architecture-overview.jpg"
style="width:750px">
 
     <h3><a id="streams_architecture_tasks" href="#streams_architecture_tasks">Stream
Partitions and Tasks</a></h3>
 
     <p>
-    The messaging layer of Kafka partitions data for storing and transporting it. Kafka Streams
partitions data for processing it.
-    In both cases, this partitioning is what enables data locality, elasticity, scalability,
high performance, and fault tolerance.
-    Kafka Streams uses the concepts of <b>partitions</b> and <b>tasks</b>
as logical units of its parallelism model based on Kafka topic partitions.
-    There are close links between Kafka Streams and Kafka in the context of parallelism:
+        The messaging layer of Kafka partitions data for storing and transporting it. Kafka
Streams partitions data for processing it.
+        In both cases, this partitioning is what enables data locality, elasticity, scalability,
high performance, and fault tolerance.
+        Kafka Streams uses the concepts of <b>partitions</b> and <b>tasks</b>
as logical units of its parallelism model based on Kafka topic partitions.
+        There are close links between Kafka Streams and Kafka in the context of parallelism:
     </p>
 
     <ul>
@@ -44,23 +44,23 @@
     </ul>
 
     <p>
-    An application's processor topology is scaled by breaking it into multiple tasks.
-    More specifically, Kafka Streams creates a fixed number of tasks based on the input stream
partitions for the application,
-    with each task assigned a list of partitions from the input streams (i.e., Kafka topics).
The assignment of partitions to tasks
-    never changes so that each task is a fixed unit of parallelism of the application. Tasks
can then instantiate their own processor topology
-    based on the assigned partitions; they also maintain a buffer for each of its assigned
partitions and process messages one-at-a-time from
-    these record buffers. As a result stream tasks can be processed independently and in
parallel without manual intervention.
+        An application's processor topology is scaled by breaking it into multiple tasks.
+        More specifically, Kafka Streams creates a fixed number of tasks based on the input
stream partitions for the application,
+        with each task assigned a list of partitions from the input streams (i.e., Kafka
topics). The assignment of partitions to tasks
+        never changes so that each task is a fixed unit of parallelism of the application.
Tasks can then instantiate their own processor topology
+        based on the assigned partitions; they also maintain a buffer for each of its assigned
partitions and process messages one-at-a-time from
+        these record buffers. As a result stream tasks can be processed independently and
in parallel without manual intervention.
     </p>
 
     <p>
-    It is important to understand that Kafka Streams is not a resource manager, but a library
that "runs" anywhere its stream processing application runs.
-    Multiple instances of the application are executed either on the same machine, or spread
across multiple machines and tasks can be distributed automatically
-    by the library to those running application instances. The assignment of partitions to
tasks never changes; if an application instance fails, all its assigned
-    tasks will be automatically restarted on other instances and continue to consume from
the same stream partitions.
+        It is important to understand that Kafka Streams is not a resource manager, but a
library that "runs" anywhere its stream processing application runs.
+        Multiple instances of the application are executed either on the same machine, or
spread across multiple machines and tasks can be distributed automatically
+        by the library to those running application instances. The assignment of partitions
to tasks never changes; if an application instance fails, all its assigned
+        tasks will be automatically restarted on other instances and continue to consume
from the same stream partitions.
     </p>
 
     <p>
-    The following diagram shows two tasks each assigned with one partition of the input streams.
+        The following diagram shows two tasks each assigned with one partition of the input
streams.
     </p>
     <img class="centered" src="/{{version}}/images/streams-architecture-tasks.jpg" style="width:400px">
     <br>
@@ -68,39 +68,39 @@
     <h3><a id="streams_architecture_threads" href="#streams_architecture_threads">Threading
Model</a></h3>
 
     <p>
-    Kafka Streams allows the user to configure the number of <b>threads</b> that
the library can use to parallelize processing within an application instance.
-    Each thread can execute one or more tasks with their processor topologies independently.
For example, the following diagram shows one stream thread running two stream tasks.
+        Kafka Streams allows the user to configure the number of <b>threads</b>
that the library can use to parallelize processing within an application instance.
+        Each thread can execute one or more tasks with their processor topologies independently.
For example, the following diagram shows one stream thread running two stream tasks.
     </p>
     <img class="centered" src="/{{version}}/images/streams-architecture-threads.jpg" style="width:400px">
 
     <p>
-    Starting more stream threads or more instances of the application merely amounts to replicating
the topology and having it process a different subset of Kafka partitions, effectively parallelizing
processing.
-    It is worth noting that there is no shared state amongst the threads, so no inter-thread
coordination is necessary. This makes it very simple to run topologies in parallel across
the application instances and threads.
-    The assignment of Kafka topic partitions amongst the various stream threads is transparently
handled by Kafka Streams leveraging <a href="https://cwiki.apache.org/confluence/display/KAFKA/Kafka+Client-side+Assignment+Proposal">Kafka's
coordination</a> functionality.
+        Starting more stream threads or more instances of the application merely amounts
to replicating the topology and having it process a different subset of Kafka partitions,
effectively parallelizing processing.
+        It is worth noting that there is no shared state amongst the threads, so no inter-thread
coordination is necessary. This makes it very simple to run topologies in parallel across
the application instances and threads.
+        The assignment of Kafka topic partitions amongst the various stream threads is transparently
handled by Kafka Streams leveraging <a href="https://cwiki.apache.org/confluence/display/KAFKA/Kafka+Client-side+Assignment+Proposal">Kafka's
coordination</a> functionality.
     </p>
 
     <p>
-    As we described above, scaling your stream processing application with Kafka Streams
is easy: you merely need to start additional instances of your application,
-    and Kafka Streams takes care of distributing partitions amongst tasks that run in the
application instances. You can start as many threads of the application
-    as there are input Kafka topic partitions so that, across all running instances of an
application, every thread (or rather, the tasks it runs) has at least one input partition
to process.
+        As we described above, scaling your stream processing application with Kafka Streams
is easy: you merely need to start additional instances of your application,
+        and Kafka Streams takes care of distributing partitions amongst tasks that run in
the application instances. You can start as many threads of the application
+        as there are input Kafka topic partitions so that, across all running instances of
an application, every thread (or rather, the tasks it runs) has at least one input partition
to process.
     </p>
     <br>
 
     <h3><a id="streams_architecture_state" href="#streams_architecture_state">Local
State Stores</a></h3>
 
     <p>
-    Kafka Streams provides so-called <b>state stores</b>, which can be used by
stream processing applications to store and query data,
-    which is an important capability when implementing stateful operations. The <a href="/{{version}}/documentation/streams/developer-guide#streams_dsl">Kafka
Streams DSL</a>, for example, automatically creates
-    and manages such state stores when you are calling stateful operators such as <code>join()</code>
or <code>aggregate()</code>, or when you are windowing a stream.
+        Kafka Streams provides so-called <b>state stores</b>, which can be used
by stream processing applications to store and query data,
+        which is an important capability when implementing stateful operations. The <a
href="/{{version}}/documentation/streams/developer-guide#streams_dsl">Kafka Streams DSL</a>,
for example, automatically creates
+        and manages such state stores when you are calling stateful operators such as <code>join()</code>
or <code>aggregate()</code>, or when you are windowing a stream.
     </p>
 
     <p>
-    Every stream task in a Kafka Streams application may embed one or more local state stores
that can be accessed via APIs to store and query data required for processing.
-    Kafka Streams offers fault-tolerance and automatic recovery for such local state stores.
+        Every stream task in a Kafka Streams application may embed one or more local state
stores that can be accessed via APIs to store and query data required for processing.
+        Kafka Streams offers fault-tolerance and automatic recovery for such local state
stores.
     </p>
 
     <p>
-    The following diagram shows two stream tasks with their dedicated local state stores.
+        The following diagram shows two stream tasks with their dedicated local state stores.
     </p>
     <img class="centered" src="/{{version}}/images/streams-architecture-states.jpg" style="width:400px">
     <br>
@@ -108,25 +108,25 @@
     <h3><a id="streams_architecture_recovery" href="#streams_architecture_recovery">Fault
Tolerance</a></h3>
 
     <p>
-    Kafka Streams builds on fault-tolerance capabilities integrated natively within Kafka.
Kafka partitions are highly available and replicated; so when stream data is persisted to
Kafka it is available
-    even if the application fails and needs to re-process it. Tasks in Kafka Streams leverage
the fault-tolerance capability
-    offered by the <a href="https://www.confluent.io/blog/tutorial-getting-started-with-the-new-apache-kafka-0.9-consumer-client/">Kafka
consumer client</a> to handle failures.
-    If a task runs on a machine that fails, Kafka Streams automatically restarts the task
in one of the remaining running instances of the application.
+        Kafka Streams builds on fault-tolerance capabilities integrated natively within Kafka.
Kafka partitions are highly available and replicated; so when stream data is persisted to
Kafka it is available
+        even if the application fails and needs to re-process it. Tasks in Kafka Streams
leverage the fault-tolerance capability
+        offered by the <a href="https://www.confluent.io/blog/tutorial-getting-started-with-the-new-apache-kafka-0.9-consumer-client/">Kafka
consumer client</a> to handle failures.
+        If a task runs on a machine that fails, Kafka Streams automatically restarts the
task in one of the remaining running instances of the application.
     </p>
 
     <p>
-    In addition, Kafka Streams makes sure that the local state stores are robust to failures,
too. For each state store, it maintains a replicated changelog Kafka topic in which it tracks
any state updates.
-    These changelog topics are partitioned as well so that each local state store instance,
and hence the task accessing the store, has its own dedicated changelog topic partition.
-    <a href="/{{version}}/documentation/#compaction">Log compaction</a> is enabled
on the changelog topics so that old data can be purged safely to prevent the topics from growing
indefinitely.
-    If tasks run on a machine that fails and are restarted on another machine, Kafka Streams
guarantees to restore their associated state stores to the content before the failure by
-    replaying the corresponding changelog topics prior to resuming the processing on the
newly started tasks. As a result, failure handling is completely transparent to the end user.
+        In addition, Kafka Streams makes sure that the local state stores are robust to failures,
too. For each state store, it maintains a replicated changelog Kafka topic in which it tracks
any state updates.
+        These changelog topics are partitioned as well so that each local state store instance,
and hence the task accessing the store, has its own dedicated changelog topic partition.
+        <a href="/{{version}}/documentation/#compaction">Log compaction</a> is
enabled on the changelog topics so that old data can be purged safely to prevent the topics
from growing indefinitely.
+        If tasks run on a machine that fails and are restarted on another machine, Kafka
Streams guarantees to restore their associated state stores to the content before the failure
by
+        replaying the corresponding changelog topics prior to resuming the processing on
the newly started tasks. As a result, failure handling is completely transparent to the end
user.
     </p>
 
     <p>
-    Note that the cost of task (re)initialization typically depends primarily on the time
for restoring the state by replaying the state stores' associated changelog topics.
-    To minimize this restoration time, users can configure their applications to have <b>standby
replicas</b> of local states (i.e. fully replicated copies of the state).
-    When a task migration happens, Kafka Streams then attempts to assign a task to an application
instance where such a standby replica already exists in order to minimize
-    the task (re)initialization cost. See <code>num.standby.replicas</code> in
the <a href="/{{version}}/documentation/#streamsconfigs"><b>Kafka Streams Configs</b></a>
Section.
+        Note that the cost of task (re)initialization typically depends primarily on the
time for restoring the state by replaying the state stores' associated changelog topics.
+        To minimize this restoration time, users can configure their applications to have
<b>standby replicas</b> of local states (i.e. fully replicated copies of the state).
+        When a task migration happens, Kafka Streams then attempts to assign a task to an
application instance where such a standby replica already exists in order to minimize
+        the task (re)initialization cost. See <code>num.standby.replicas</code>
in the <a href="/{{version}}/documentation/#streamsconfigs"><b>Kafka Streams Configs</b></a>
Section.
     </p>
 
     <div class="pagination">
@@ -138,9 +138,9 @@
 <!--#include virtual="../../includes/_header.htm" -->
 <!--#include virtual="../../includes/_top.htm" -->
 <div class="content documentation documentation--current">
-	<!--#include virtual="../../includes/_nav.htm" -->
-	<div class="right">
-		<!--#include virtual="../../includes/_docs_banner.htm" -->
+    <!--#include virtual="../../includes/_nav.htm" -->
+    <div class="right">
+        <!--#include virtual="../../includes/_docs_banner.htm" -->
         <ul class="breadcrumbs">
             <li><a href="/documentation">Documentation</a></li>
             <li><a href="/documentation/streams">Streams</a></li>

http://git-wip-us.apache.org/repos/asf/kafka-site/blob/422095a8/0110/streams/core-concepts.html
----------------------------------------------------------------------
diff --git a/0110/streams/core-concepts.html b/0110/streams/core-concepts.html
index c0934ad..b50495d 100644
--- a/0110/streams/core-concepts.html
+++ b/0110/streams/core-concepts.html
@@ -143,9 +143,9 @@
 <!--#include virtual="../../includes/_header.htm" -->
 <!--#include virtual="../../includes/_top.htm" -->
 <div class="content documentation documentation--current">
-	<!--#include virtual="../../includes/_nav.htm" -->
-	<div class="right">
-		<!--#include virtual="../../includes/_docs_banner.htm" -->
+    <!--#include virtual="../../includes/_nav.htm" -->
+    <div class="right">
+        <!--#include virtual="../../includes/_docs_banner.htm" -->
         <ul class="breadcrumbs">
             <li><a href="/documentation">Documentation</a></li>
             <li><a href="/documentation/streams">Streams</a></li>


Mime
View raw message