http://git-wip-us.apache.org/repos/asf/kafka-site/blob/04199a2e/0102/streams/architecture.html
----------------------------------------------------------------------
diff --git a/0102/streams/architecture.html b/0102/streams/architecture.html
new file mode 100644
index 0000000..ee601f8
--- /dev/null
+++ b/0102/streams/architecture.html
@@ -0,0 +1,162 @@
+<!--
+ 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.
+-->
+
+<script><!--#include virtual="../js/templateData.js" --></script>
+
+<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>
+
+    <p>
+    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:
+    </p>
+
+    <ul>
+        <li>Each <b>stream partition</b> is a totally ordered sequence of data 
records and maps to a Kafka <b>topic partition</b>.</li>
+        <li>A <b>data record</b> in the stream maps to a Kafka <b>message</b> 
from that topic.</li>
+        <li>The <b>keys</b> of data records determine the partitioning of data 
in both Kafka and Kafka Streams, i.e., how data is routed to specific 
partitions within topics.</li>
+    </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.
+    </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.
+    </p>
+
+    <p>
+    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>
+
+    <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.
+    </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.
+    </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.
+    </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="#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.
+    </p>
+
+    <p>
+    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>
+
+    <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.
+    </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.
+    </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> at 
the <a href="/{{version}}/documentation/#streamsconfigs">Kafka Streams 
Configs</a> Section.
+    </p>
+
+    <div class="pagination">
+        <a href="/{{version}}/documentation/streams/core-concepts" 
class="pagination__btn pagination__btn__prev">Previous</a>
+        <a href="/{{version}}/documentation/streams/developer-guide" 
class="pagination__btn pagination__btn__next">Next</a>
+    </div>
+</script>
+
+<!--#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" -->
+        <ul class="breadcrumbs">
+            <li><a href="/documentation">Documentation</a></li>
+            <li><a href="/documentation/streams">Streams</a></li>
+        </ul>
+        <div class="p-content"></div>
+    </div>
+</div>
+<!--#include virtual="../../includes/_footer.htm" -->
+<script>
+$(function() {
+  // Show selected style on nav item
+  $('.b-nav__streams').addClass('selected');
+
+  // Display docs subnav items
+  $('.b-nav__docs').parent().toggleClass('nav__item__with__subs--expanded');
+});
+</script>

http://git-wip-us.apache.org/repos/asf/kafka-site/blob/04199a2e/0102/streams/core-concepts.html
----------------------------------------------------------------------
diff --git a/0102/streams/core-concepts.html b/0102/streams/core-concepts.html
new file mode 100644
index 0000000..1ec58ea
--- /dev/null
+++ b/0102/streams/core-concepts.html
@@ -0,0 +1,140 @@
+<!--
+ 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.
+-->
+
+<script><!--#include virtual="../js/templateData.js" --></script>
+
+<script id="content-template" type="text/x-handlebars-template">
+    <h1>Core Concepts</h1>
+    <p>
+    We first summarize the key concepts of Kafka Streams.
+    </p>
+
+    <h3><a id="streams_topology" href="#streams_topology">Stream Processing 
Topology</a></h3>
+
+    <ul>
+        <li>A <b>stream</b> is the most important abstraction provided by 
Kafka Streams: it represents an unbounded, continuously updating data set. A 
stream is an ordered, replayable, and fault-tolerant sequence of immutable data 
records, where a <b>data record</b> is defined as a key-value pair.</li>
+        <li>A <b>stream processing application</b> is any program that makes 
use of the Kafka Streams library. It defines its computational logic through 
one or more <b>processor topologies</b>, where a processor topology is a graph 
of stream processors (nodes) that are connected by streams (edges).</li>
+        <li>A <b>stream processor</b> is a node in the processor topology; it 
represents a processing step to transform data in streams by receiving one 
input record at a time from its upstream processors in the topology, applying 
its operation to it, and may subsequently produce one or more output records to 
its downstream processors. </li>
+    </ul>
+
+    There are two special processors in the topology:
+
+    <ul>
+        <li><b>Source Processor</b>: A source processor is a special type of 
stream processor that does not have any upstream processors. It produces an 
input stream to its topology from one or multiple Kafka topics by consuming 
records from these topics and forward them to its down-stream processors.</li>
+        <li><b>Sink Processor</b>: A sink processor is a special type of 
stream processor that does not have down-stream processors. It sends any 
received records from its up-stream processors to a specified Kafka topic.</li>
+    </ul>
+
+    <img class="centered" 
src="/{{version}}/images/streams-architecture-topology.jpg" style="width:400px">
+
+    <p>
+        Kafka Streams offers two ways to define the stream processing 
topology: the <a href="#streams_dsl"><b>Kafka Streams DSL</b></a> provides
+        the most common data transformation operations such as 
<code>map</code>, <code>filter</code>, <code>join</code> and 
<code>aggregations</code> out of the box; the lower-level <a 
href="#streams_processor"><b>Processor API</b></a> allows
+        developers define and connect custom processors as well as to interact 
with <a href="#streams_state">state stores</a>.
+    </p>
+
+    <p>
+        A processor topology is merely a logical abstraction for your stream 
processing code.
+        At runtime, the logical topology is instantiated and replicated inside 
the application for parallel processing (see <a 
href="#streams_architecture_tasks">Stream Partitions and Tasks</a> for details).
+    </p>
+
+    <h3><a id="streams_time" href="#streams_time">Time</a></h3>
+
+    <p>
+        A critical aspect in stream processing is the notion of <b>time</b>, 
and how it is modeled and integrated.
+        For example, some operations such as <b>windowing</b> are defined 
based on time boundaries.
+    </p>
+    <p>
+        Common notions of time in streams are:
+    </p>
+
+    <ul>
+        <li><b>Event time</b> - The point in time when an event or data record 
occurred, i.e. was originally created "at the source". <b>Example:</b> If the 
event is a geo-location change reported by a GPS sensor in a car, then the 
associated event-time would be the time when the GPS sensor captured the 
location change.</li>
+        <li><b>Processing time</b> - The point in time when the event or data 
record happens to be processed by the stream processing application, i.e. when 
the record is being consumed. The processing time may be milliseconds, hours, 
or days etc. later than the original event time. <b>Example:</b> Imagine an 
analytics application that reads and processes the geo-location data reported 
from car sensors to present it to a fleet management dashboard. Here, 
processing-time in the analytics application might be milliseconds or seconds 
(e.g. for real-time pipelines based on Apache Kafka and Kafka Streams) or hours 
(e.g. for batch pipelines based on Apache Hadoop or Apache Spark) after 
event-time.</li>
+        <li><b>Ingestion time</b> - The point in time when an event or data 
record is stored in a topic partition by a Kafka broker. The difference to 
event time is that this ingestion timestamp is generated when the record is 
appended to the target topic by the Kafka broker, not when the record is 
created "at the source". The difference to processing time is that processing 
time is when the stream processing application processes the record. <b>For 
example,</b> if a record is never processed, there is no notion of processing 
time for it, but it still has an ingestion time.</li>
+    </ul>
+    <p>
+        The choice between event-time and ingestion-time is actually done 
through the configuration of Kafka (not Kafka Streams): From Kafka 0.10.x 
onwards, timestamps are automatically embedded into Kafka messages. Depending 
on Kafka's configuration these timestamps represent event-time or 
ingestion-time. The respective Kafka configuration setting can be specified on 
the broker level or per topic. The default timestamp extractor in Kafka Streams 
will retrieve these embedded timestamps as-is. Hence, the effective time 
semantics of your application depend on the effective Kafka configuration for 
these embedded timestamps.
+    </p>
+    <p>
+        Kafka Streams assigns a <b>timestamp</b> to every data record
+        via the <code>TimestampExtractor</code> interface.
+        Concrete implementations of this interface may retrieve or compute 
timestamps based on the actual contents of data records such as an embedded 
timestamp field
+        to provide event-time semantics, or use any other approach such as 
returning the current wall-clock time at the time of processing,
+        thereby yielding processing-time semantics to stream processing 
applications.
+        Developers can thus enforce different notions of time depending on 
their business needs. For example,
+        per-record timestamps describe the progress of a stream with regards 
to time (although records may be out-of-order within the stream) and
+        are leveraged by time-dependent operations such as joins.
+    </p>
+
+    <p>
+        Finally, whenever a Kafka Streams application writes records to Kafka, 
then it will also assign timestamps to these new records. The way the 
timestamps are assigned depends on the context:
+    </p>
+
+    <ul>
+        <li> When new output records are generated via processing some input 
record, for example, <code>context.forward()</code> triggered in the 
<code>process()</code> function call, output record timestamps are inherited 
from input record timestamps directly.</li>
+        <li> When new output records are generated via periodic functions such 
as <code>punctuate()</code>, the output record timestamp is defined as the 
current internal time (obtained through <code>context.timestamp()</code>) of 
the stream task.</li>
+        <li> For aggregations, the timestamp of a resulting aggregate update 
record will be that of the latest arrived input record that triggered the 
update.</li>
+    </ul>
+
+    <h3><a id="streams_state" href="#streams_state">States</a></h3>
+
+    <p>
+        Some stream processing applications don't require state, which means 
the processing of a message is independent from
+        the processing of all other messages.
+        However, being able to maintain state opens up many possibilities for 
sophisticated stream processing applications: you
+        can join input streams, or group and aggregate data records. Many such 
stateful operators are provided by the <a href="#streams_dsl"><b>Kafka Streams 
DSL</b></a>.
+    </p>
+    <p>
+        Kafka Streams provides so-called <b>state stores</b>, which can be 
used by stream processing applications to store and query data.
+        This is an important capability when implementing stateful operations.
+        Every task in Kafka Streams embeds one or more state stores that can 
be accessed via APIs to store and query data required for processing.
+        These state stores can either be a persistent key-value store, an 
in-memory hashmap, or another convenient data structure.
+        Kafka Streams offers fault-tolerance and automatic recovery for local 
state stores.
+    </p>
+    <p>
+        Kafka Streams allows direct read-only queries of the state stores by 
methods, threads, processes or applications external to the stream processing 
application that created the state stores. This is provided through a feature 
called <b>Interactive Queries</b>. All stores are named and Interactive Queries 
exposes only the read operations of the underlying implementation.
+    </p>
+
+    <div class="pagination">
+        <a href="/{{version}}/documentation/streams" class="pagination__btn 
pagination__btn__prev">Previous</a>
+        <a href="/{{version}}/documentation/streams/architecture" 
class="pagination__btn pagination__btn__next">Next</a>
+    </div>
+</script>
+
+<!--#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" -->
+        <ul class="breadcrumbs">
+            <li><a href="/documentation">Documentation</a></li>
+            <li><a href="/documentation/streams">Streams</a></li>
+        </ul>
+        <div class="p-content"></div>
+    </div>
+</div>
+<!--#include virtual="../../includes/_footer.htm" -->
+<script>
+$(function() {
+  // Show selected style on nav item
+  $('.b-nav__streams').addClass('selected');
+
+  // Display docs subnav items
+  $('.b-nav__docs').parent().toggleClass('nav__item__with__subs--expanded');
+});
+</script>

http://git-wip-us.apache.org/repos/asf/kafka-site/blob/04199a2e/0102/streams/developer-guide.html
----------------------------------------------------------------------
diff --git a/0102/streams/developer-guide.html 
b/0102/streams/developer-guide.html
new file mode 100644
index 0000000..ddadf10
--- /dev/null
+++ b/0102/streams/developer-guide.html
@@ -0,0 +1,577 @@
+<!--
+ 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.
+-->
+
+<script><!--#include virtual="../js/templateData.js" --></script>
+
+<script id="content-template" type="text/x-handlebars-template">
+    <h1>Developer Guide</h1>
+
+    <p>
+    There is a <a 
href="/{{version}}/documentation/#quickstart_kafkastreams">quickstart</a> 
example that provides how to run a stream processing program coded in the Kafka 
Streams library.
+    This section focuses on how to write, configure, and execute a Kafka 
Streams application.
+    </p>
+
+    <p>
+    As we have mentioned above, the computational logic of a Kafka Streams 
application is defined as a <a href="#streams_topology">processor topology</a>.
+    Currently Kafka Streams provides two sets of APIs to define the processor 
topology, which will be described in the subsequent sections.
+    </p>
+
+    <h3><a id="streams_processor" href="#streams_processor">Low-Level 
Processor API</a></h3>
+
+    <h4><a id="streams_processor_process" 
href="#streams_processor_process">Processor</a></h4>
+
+    <p>
+    As mentioned in the <a href="#streams_concepts">Core Concepts</a> section, 
a stream processor is a node in the processor topology that represents a single 
processing step.
+    With the <code>Processor</code> API developers can define arbitrary stream 
processors that process one received record at a time, and connect these 
processors with
+    their associated state stores to compose the processor topology that 
represents their customized processing logic.
+    </p>
+
+    <p>
+    The <code>Processor</code> interface provides two main API methods:
+    <code>process</code> and <code>punctuate</code>. The <code>process</code> 
method is performed on each
+    of the received record; and the <code>punctuate</code> method is performed 
periodically based on elapsed time.
+    In addition, the processor can maintain the current 
<code>ProcessorContext</code> instance variable initialized in the
+    <code>init</code> method, and use the context to schedule the punctuation 
period (<code>context().schedule</code>), to
+    forward the modified / new key-value pair to downstream processors 
(<code>context().forward</code>), to commit the current
+    processing progress (<code>context().commit</code>), etc.
+    </p>
+
+    <p>
+    The following example <code>Processor</code> implementation defines a 
simple word-count algorithm:
+    </p>
+
+    <pre>
+    public class MyProcessor implements Processor&lt;String, String&gt; {
+    private ProcessorContext context;
+    private KeyValueStore&lt;String, Long&gt; kvStore;
+
+    @Override
+    @SuppressWarnings("unchecked")
+    public void init(ProcessorContext context) {
+    // keep the processor context locally because we need it in punctuate() 
and commit()
+    this.context = context;
+
+    // call this processor's punctuate() method every 1000 milliseconds.
+    this.context.schedule(1000);
+
+    // retrieve the key-value store named "Counts"
+    this.kvStore = (KeyValueStore&lt;String, Long&gt;) 
context.getStateStore("Counts");
+    }
+
+    @Override
+    public void process(String dummy, String line) {
+    String[] words = line.toLowerCase().split(" ");
+
+    for (String word : words) {
+        Long oldValue = this.kvStore.get(word);
+
+        if (oldValue == null) {
+            this.kvStore.put(word, 1L);
+        } else {
+            this.kvStore.put(word, oldValue + 1L);
+        }
+    }
+    }
+
+    @Override
+    public void punctuate(long timestamp) {
+    KeyValueIterator&lt;String, Long&gt; iter = this.kvStore.all();
+
+    while (iter.hasNext()) {
+        KeyValue&lt;String, Long&gt; entry = iter.next();
+        context.forward(entry.key, entry.value.toString());
+    }
+
+    iter.close();
+    // commit the current processing progress
+    context.commit();
+    }
+
+    @Override
+    public void close() {
+    // close any resources managed by this processor.
+    // Note: Do not close any StateStores as these are managed
+    // by the library
+    }
+    };
+    </pre>
+
+    <p>
+    In the above implementation, the following actions are performed:
+    </p>
+
+    <ul>
+        <li>In the <code>init</code> method, schedule the punctuation every 1 
second and retrieve the local state store by its name "Counts".</li>
+        <li>In the <code>process</code> method, upon each received record, 
split the value string into words, and update their counts into the state store 
(we will talk about this feature later in the section).</li>
+        <li>In the <code>punctuate</code> method, iterate the local state 
store and send the aggregated counts to the downstream processor, and commit 
the current stream state.</li>
+    </ul>
+
+
+    <h4><a id="streams_processor_topology" 
href="#streams_processor_topology">Processor Topology</a></h4>
+
+    <p>
+    With the customized processors defined in the Processor API, developers 
can use the <code>TopologyBuilder</code> to build a processor topology
+    by connecting these processors together:
+    </p>
+
+    <pre>
+    TopologyBuilder builder = new TopologyBuilder();
+
+    builder.addSource("SOURCE", "src-topic")
+    // add "PROCESS1" node which takes the source processor "SOURCE" as its 
upstream processor
+    .addProcessor("PROCESS1", () -> new MyProcessor1(), "SOURCE")
+
+    // add "PROCESS2" node which takes "PROCESS1" as its upstream processor
+    .addProcessor("PROCESS2", () -> new MyProcessor2(), "PROCESS1")
+
+    // add "PROCESS3" node which takes "PROCESS1" as its upstream processor
+    .addProcessor("PROCESS3", () -> new MyProcessor3(), "PROCESS1")
+
+    // add the sink processor node "SINK1" that takes Kafka topic "sink-topic1"
+    // as output and the "PROCESS1" node as its upstream processor
+    .addSink("SINK1", "sink-topic1", "PROCESS1")
+
+    // add the sink processor node "SINK2" that takes Kafka topic "sink-topic2"
+    // as output and the "PROCESS2" node as its upstream processor
+    .addSink("SINK2", "sink-topic2", "PROCESS2")
+
+    // add the sink processor node "SINK3" that takes Kafka topic "sink-topic3"
+    // as output and the "PROCESS3" node as its upstream processor
+    .addSink("SINK3", "sink-topic3", "PROCESS3");
+    </pre>
+
+    There are several steps in the above code to build the topology, and here 
is a quick walk through:
+
+    <ul>
+        <li>First of all a source node named "SOURCE" is added to the topology 
using the <code>addSource</code> method, with one Kafka topic "src-topic" fed 
to it.</li>
+        <li>Three processor nodes are then added using the 
<code>addProcessor</code> method; here the first processor is a child of the 
"SOURCE" node, but is the parent of the other two processors.</li>
+        <li>Finally three sink nodes are added to complete the topology using 
the <code>addSink</code> method, each piping from a different parent processor 
node and writing to a separate topic.</li>
+    </ul>
+
+    <h4><a id="streams_processor_statestore" 
href="#streams_processor_statestore">State Stores</a></h4>
+
+    <p>
+    Note that the <code>Processor</code> API is not limited to only accessing 
the current records as they arrive in the <code>process()</code> method, but 
can also maintain processing states
+    that keep recently arrived records to use in stateful processing 
operations such as windowed joins or aggregation.
+    To take advantage of these states, users can define a state store by 
implementing the <code>StateStore</code> interface (the Kafka Streams library 
also has a few extended interfaces such as <code>KeyValueStore</code>);
+    in practice, though, users usually do not need to customize such a state 
store from scratch but can simply use the <code>Stores</code> factory to define 
a state store by specifying whether it should be persistent, log-backed, etc.
+    In the following example, a persistent key-value store named “Counts” 
with key type <code>String</code> and value type <code>Long</code> is created.
+    </p>
+
+    <pre>
+    StateStoreSupplier countStore = Stores.create("Counts")
+    .withKeys(Serdes.String())
+    .withValues(Serdes.Long())
+    .persistent()
+    .build();
+    </pre>
+
+    <p>
+    To take advantage of these state stores, developers can use the 
<code>TopologyBuilder.addStateStore</code> method when building the
+    processor topology to create the local state and associate it with the 
processor nodes that needs to access it; or they can connect a created
+    state store with the existing processor nodes through 
<code>TopologyBuilder.connectProcessorAndStateStores</code>.
+    </p>
+
+    <pre>
+    TopologyBuilder builder = new TopologyBuilder();
+
+    builder.addSource("SOURCE", "src-topic")
+
+    .addProcessor("PROCESS1", MyProcessor1::new, "SOURCE")
+    // add the created state store "COUNTS" associated with processor 
"PROCESS1"
+    .addStateStore(countStore, "PROCESS1")
+    .addProcessor("PROCESS2", MyProcessor3::new /* the ProcessorSupplier that 
can generate MyProcessor3 */, "PROCESS1")
+    .addProcessor("PROCESS3", MyProcessor3::new /* the ProcessorSupplier that 
can generate MyProcessor3 */, "PROCESS1")
+
+    // connect the state store "COUNTS" with processor "PROCESS2"
+    .connectProcessorAndStateStores("PROCESS2", "COUNTS");
+
+    .addSink("SINK1", "sink-topic1", "PROCESS1")
+    .addSink("SINK2", "sink-topic2", "PROCESS2")
+    .addSink("SINK3", "sink-topic3", "PROCESS3");
+    </pre>
+
+    In the next section we present another way to build the processor 
topology: the Kafka Streams DSL.
+    <br>
+
+    <h3><a id="streams_dsl" href="#streams_dsl">High-Level Streams DSL</a></h3>
+
+    To build a processor topology using the Streams DSL, developers can apply 
the <code>KStreamBuilder</code> class, which is extended from the 
<code>TopologyBuilder</code>.
+    A simple example is included with the source code for Kafka in the 
<code>streams/examples</code> package. The rest of this section will walk
+    through some code to demonstrate the key steps in creating a topology 
using the Streams DSL, but we recommend developers to read the full example 
source
+    codes for details.
+
+    <h4><a id="streams_duality" href="#streams_duality">Duality of Streams and 
Tables</a></h4>
+
+    <p>
+    Before we discuss concepts such as aggregations in Kafka Streams we must 
first introduce tables, and most importantly the relationship between tables 
and streams:
+    the so-called <a 
href="https://engineering.linkedin.com/distributed-systems/log-what-every-software-engineer-should-know-about-real-time-datas-unifying/";>stream-table
 duality</a>.
+    Essentially, this duality means that a stream can be viewed as a table, 
and vice versa. Kafka's log compaction feature, for example, exploits this 
duality.
+    </p>
+
+    <p>
+    A simple form of a table is a collection of key-value pairs, also called a 
map or associative array. Such a table may look as follows:
+    </p>
+    <img class="centered" 
src="/{{version}}/images/streams-table-duality-01.png">
+
+    The <b>stream-table duality</b> describes the close relationship between 
streams and tables.
+    <ul>
+    <li><b>Stream as Table</b>: A stream can be considered a changelog of a 
table, where each data record in the stream captures a state change of the 
table. A stream is thus a table in disguise, and it can be easily turned into a 
“real” table by replaying the changelog from beginning to end to 
reconstruct the table. Similarly, in a more general analogy, aggregating data 
records in a stream – such as computing the total number of pageviews by user 
from a stream of pageview events – will return a table (here with the key and 
the value being the user and its corresponding pageview count, 
respectively).</li>
+    <li><b>Table as Stream</b>: A table can be considered a snapshot, at a 
point in time, of the latest value for each key in a stream (a stream's data 
records are key-value pairs). A table is thus a stream in disguise, and it can 
be easily turned into a “real” stream by iterating over each key-value 
entry in the table.</li>
+    </ul>
+
+    <p>
+    Let's illustrate this with an example. Imagine a table that tracks the 
total number of pageviews by user (first column of diagram below). Over time, 
whenever a new pageview event is processed, the state of the table is updated 
accordingly. Here, the state changes between different points in time – and 
different revisions of the table – can be represented as a changelog stream 
(second column).
+    </p>
+    <img class="centered" 
src="/{{version}}/images/streams-table-duality-02.png" style="width:300px">
+
+    <p>
+    Interestingly, because of the stream-table duality, the same stream can be 
used to reconstruct the original table (third column):
+    </p>
+    <img class="centered" 
src="/{{version}}/images/streams-table-duality-03.png" style="width:600px">
+
+    <p>
+    The same mechanism is used, for example, to replicate databases via change 
data capture (CDC) and, within Kafka Streams, to replicate its so-called state 
stores across machines for fault-tolerance.
+    The stream-table duality is such an important concept that Kafka Streams 
models it explicitly via the <a href="#streams_kstream_ktable">KStream, KTable, 
and GlobalKTable</a> interfaces, which we describe in the next sections.
+    </p>
+
+    <h5><a id="streams_kstream_ktable" href="#streams_kstream_ktable">KStream, 
KTable, and GlobalKTable</a></h5>
+    The DSL uses three main abstractions. A <b>KStream</b> is an abstraction 
of a record stream, where each data record represents a self-contained datum in 
the unbounded data set.
+    A <b>KTable</b> is an abstraction of a changelog stream, where each data 
record represents an update. More precisely, the value in a data record is 
considered to be an update of the last value for the same record key,
+    if any (if a corresponding key doesn't exist yet, the update will be 
considered a create).
+    Like a <b>KTable</b>, a <b>GlobalKTable</b> is an abstraction of a 
changelog stream, where each data record represents an update.
+    However, a <b>GlobalKTable</b> is different from a <b>KTable</b> in that 
it is fully replicated on each KafkaStreams instance.
+    <b>GlobalKTable</b> also provides the ability to look up current values of 
data records by keys.
+    This table-lookup functionality is available through <a 
href="#streams_dsl_joins">join operations</a>.
+
+    To illustrate the difference between KStreams and KTables/GlobalKTables, 
let’s imagine the following two data records are being sent to the stream:
+
+    <pre>
+    ("alice", 1) --> ("alice", 3)
+    </pre>
+
+    If these records a KStream and the stream processing application were to 
sum the values it would return <code>4</code>. If these records were a KTable 
or GlobalKTable, the return would be <code>3</code>, since the last record 
would be considered as an update.
+
+    <h4><a id="streams_dsl_source" href="#streams_dsl_source">Create Source 
Streams from Kafka</a></h4>
+
+    <p>
+    Either a <b>record stream</b> (defined as <code>KStream</code>) or a 
<b>changelog stream</b> (defined as <code>KTable</code> or 
<code>GlobalKTable</code>)
+    can be created as a source stream from one or more Kafka topics (for 
<code>KTable</code> and <code>GlobalKTable</code> you can only create the 
source stream
+    from a single topic).
+    </p>
+
+    <pre>
+    KStreamBuilder builder = new KStreamBuilder();
+
+    KStream&lt;String, GenericRecord&gt; source1 = builder.stream("topic1", 
"topic2");
+    KTable&lt;String, GenericRecord&gt; source2 = builder.table("topic3", 
"stateStoreName");
+    GlobalKTable&lt;String, GenericRecord&gt; source2 = 
builder.globalTable("topic4", "globalStoreName");
+    </pre>
+
+    <h4><a id="streams_dsl_windowing" href="#streams_dsl_windowing">Windowing 
a stream</a></h4>
+    A stream processor may need to divide data records into time buckets, i.e. 
to <b>window</b> the stream by time. This is usually needed for join and 
aggregation operations, etc. Kafka Streams currently defines the following 
types of windows:
+    <ul>
+    <li><b>Hopping time windows</b> are windows based on time intervals. They 
model fixed-sized, (possibly) overlapping windows. A hopping window is defined 
by two properties: the window's size and its advance interval (aka "hop"). The 
advance interval specifies by how much a window moves forward relative to the 
previous one. For example, you can configure a hopping window with a size 5 
minutes and an advance interval of 1 minute. Since hopping windows can overlap 
a data record may belong to more than one such windows.</li>
+    <li><b>Tumbling time windows</b> are a special case of hopping time 
windows and, like the latter, are windows based on time intervals. They model 
fixed-size, non-overlapping, gap-less windows. A tumbling window is defined by 
a single property: the window's size. A tumbling window is a hopping window 
whose window size is equal to its advance interval. Since tumbling windows 
never overlap, a data record will belong to one and only one window.</li>
+    <li><b>Sliding windows</b> model a fixed-size window that slides 
continuously over the time axis; here, two data records are said to be included 
in the same window if the difference of their timestamps is within the window 
size. Thus, sliding windows are not aligned to the epoch, but on the data 
record timestamps. In Kafka Streams, sliding windows are used only for join 
operations, and can be specified through the <code>JoinWindows</code> 
class.</li>
+    <li><b>Session windows</b> are used to aggregate key-based events into 
sessions.
+        Sessions represent a period of activity separated by a defined gap of 
inactivity.
+        Any events processed that fall within the inactivity gap of any 
existing sessions are merged into the existing sessions.
+        If the event falls outside of the session gap, then a new session will 
be created.
+        Session windows are tracked independently across keys (e.g. windows of 
different keys typically have different start and end times) and their sizes 
vary (even windows for the same key typically have different sizes);
+        as such session windows can't be pre-computed and are instead derived 
from analyzing the timestamps of the data records.
+    </li>
+    </ul>
+
+    <p>
+    In the Kafka Streams DSL users can specify a <b>retention period</b> for 
the window. This allows Kafka Streams to retain old window buckets for a period 
of time in order to wait for the late arrival of records whose timestamps fall 
within the window interval.
+    If a record arrives after the retention period has passed, the record 
cannot be processed and is dropped.
+    </p>
+
+    <p>
+    Late-arriving records are always possible in real-time data streams. 
However, it depends on the effective <a href="#streams_time">time semantics</a> 
how late records are handled. Using processing-time, the semantics are “when 
the data is being processed”,
+    which means that the notion of late records is not applicable as, by 
definition, no record can be late. Hence, late-arriving records only really can 
be considered as such (i.e. as arriving “late”) for event-time or 
ingestion-time semantics. In both cases,
+    Kafka Streams is able to properly handle late-arriving records.
+    </p>
+
+    <h4><a id="streams_dsl_joins" href="#streams_dsl_joins">Join multiple 
streams</a></h4>
+    A <b>join</b> operation merges two streams based on the keys of their data 
records, and yields a new stream. A join over record streams usually needs to 
be performed on a windowing basis because otherwise the number of records that 
must be maintained for performing the join may grow indefinitely. In Kafka 
Streams, you may perform the following join operations:
+    <ul>
+    <li><b>KStream-to-KStream Joins</b> are always windowed joins, since 
otherwise the memory and state required to compute the join would grow 
infinitely in size. Here, a newly received record from one of the streams is 
joined with the other stream's records within the specified window interval to 
produce one result for each matching pair based on user-provided 
<code>ValueJoiner</code>. A new <code>KStream</code> instance representing the 
result stream of the join is returned from this operator.</li>
+
+    <li><b>KTable-to-KTable Joins</b> are join operations designed to be 
consistent with the ones in relational databases. Here, both changelog streams 
are materialized into local state stores first. When a new record is received 
from one of the streams, it is joined with the other stream's materialized 
state stores to produce one result for each matching pair based on 
user-provided ValueJoiner. A new <code>KTable</code> instance representing the 
result stream of the join, which is also a changelog stream of the represented 
table, is returned from this operator.</li>
+    <li><b>KStream-to-KTable Joins</b> allow you to perform table lookups 
against a changelog stream (<code>KTable</code>) upon receiving a new record 
from another record stream (<code>KStream</code>). An example use case would be 
to enrich a stream of user activities (<code>KStream</code>) with the latest 
user profile information (<code>KTable</code>). Only records received from the 
record stream will trigger the join and produce results via 
<code>ValueJoiner</code>, not vice versa (i.e., records received from the 
changelog stream will be used only to update the materialized state store). A 
new <code>KStream</code> instance representing the result stream of the join is 
returned from this operator.</li>
+    <li><b>KStream-to-GlobalKTable Joins</b> allow you to perform table 
lookups against a fully replicated changelog stream (<code>GlobalKTable</code>) 
upon receiving a new record from another record stream (<code>KStream</code>).
+        Joins with a <code>GlobalKTable</code> don't require repartitioning of 
the input <code>KStream</code> as all partitions of the 
<code>GlobalKTable</code> are available on every KafkaStreams instance.
+        The <code>KeyValueMapper</code> provided with the join operation is 
applied to each KStream record to extract the join-key that is used to do the 
lookup to the GlobalKTable so non-record-key joins are possible.
+        An example use case would be to enrich a stream of user activities 
(<code>KStream</code>) with the latest user profile information 
(<code>GlobalKTable</code>).
+        Only records received from the record stream will trigger the join and 
produce results via <code>ValueJoiner</code>, not vice versa (i.e., records 
received from the changelog stream will be used only to update the materialized 
state store).
+        A new <code>KStream</code> instance representing the result stream of 
the join is returned from this operator.</li>
+    </ul>
+
+    Depending on the operands the following join operations are supported: 
<b>inner joins</b>, <b>outer joins</b> and <b>left joins</b>.
+    Their <a 
href="https://cwiki.apache.org/confluence/display/KAFKA/Kafka+Streams+Join+Semantics";>semantics</a>
 are similar to the corresponding operators in relational databases.
+
+    <h5><a id="streams_dsl_aggregations" 
href="#streams_dsl_aggregations">Aggregate a stream</a></h5>
+    An <b>aggregation</b> operation takes one input stream, and yields a new 
stream by combining multiple input records into a single output record. 
Examples of aggregations are computing counts or sum. An aggregation over 
record streams usually needs to be performed on a windowing basis because 
otherwise the number of records that must be maintained for performing the 
aggregation may grow indefinitely.
+
+    <p>
+    In the Kafka Streams DSL, an input stream of an aggregation can be a 
<code>KStream</code> or a <code>KTable</code>, but the output stream will 
always be a <code>KTable</code>.
+    This allows Kafka Streams to update an aggregate value upon the late 
arrival of further records after the value was produced and emitted.
+    When such late arrival happens, the aggregating <code>KStream</code> or 
<code>KTable</code> simply emits a new aggregate value. Because the output is a 
<code>KTable</code>, the new value is considered to overwrite the old value 
with the same key in subsequent processing steps.
+    </p>
+
+    <h4><a id="streams_dsl_transform" href="#streams_dsl_transform">Transform 
a stream</a></h4>
+
+    <p>
+    Besides join and aggregation operations, there is a list of other 
transformation operations provided for <code>KStream</code> and 
<code>KTable</code> respectively.
+    Each of these operations may generate either one or more 
<code>KStream</code> and <code>KTable</code> objects and
+    can be translated into one or more connected processors into the 
underlying processor topology.
+    All these transformation methods can be chained together to compose a 
complex processor topology.
+    Since <code>KStream</code> and <code>KTable</code> are strongly typed, all 
these transformation operations are defined as
+    generics functions where users could specify the input and output data 
types.
+    </p>
+
+    <p>
+    Among these transformations, <code>filter</code>, <code>map</code>, 
<code>mapValues</code>, etc, are stateless
+    transformation operations and can be applied to both <code>KStream</code> 
and <code>KTable</code>,
+    where users can usually pass a customized function to these functions as a 
parameter, such as <code>Predicate</code> for <code>filter</code>,
+    <code>KeyValueMapper</code> for <code>map</code>, etc:
+
+    </p>
+
+    <pre>
+    // written in Java 8+, using lambda expressions
+    KStream&lt;String, GenericRecord&gt; mapped = source1.mapValue(record -> 
record.get("category"));
+    </pre>
+
+    <p>
+    Stateless transformations, by definition, do not depend on any state for 
processing, and hence implementation-wise
+    they do not require a state store associated with the stream processor; 
Stateful transformations, on the other hand,
+    require accessing an associated state for processing and producing outputs.
+    For example, in <code>join</code> and <code>aggregate</code> operations, a 
windowing state is usually used to store all the received records
+    within the defined window boundary so far. The operators can then access 
these accumulated records in the store and compute
+    based on them.
+    </p>
+
+    <pre>
+    // written in Java 8+, using lambda expressions
+    KTable&lt;Windowed&lt;String&gt;, Long&gt; counts = 
source1.groupByKey().aggregate(
+    () -> 0L,  // initial value
+    (aggKey, value, aggregate) -> aggregate + 1L,   // aggregating value
+    TimeWindows.of("counts", 5000L).advanceBy(1000L), // intervals in 
milliseconds
+    Serdes.Long() // serde for aggregated value
+    );
+
+    KStream&lt;String, String&gt; joined = source1.leftJoin(source2,
+    (record1, record2) -> record1.get("user") + "-" + record2.get("region");
+    );
+    </pre>
+
+    <h4><a id="streams_dsl_sink" href="#streams_dsl_sink">Write streams back 
to Kafka</a></h4>
+
+    <p>
+    At the end of the processing, users can choose to (continuously) write the 
final resulted streams back to a Kafka topic through
+    <code>KStream.to</code> and <code>KTable.to</code>.
+    </p>
+
+    <pre>
+    joined.to("topic4");
+    </pre>
+
+    If your application needs to continue reading and processing the records 
after they have been materialized
+    to a topic via <code>to</code> above, one option is to construct a new 
stream that reads from the output topic;
+    Kafka Streams provides a convenience method called <code>through</code>:
+
+    <pre>
+    // equivalent to
+    //
+    // joined.to("topic4");
+    // materialized = builder.stream("topic4");
+    KStream&lt;String, String&gt; materialized = joined.through("topic4");
+    </pre>
+    <br>
+
+    <h3><a id="streams_execute" href="#streams_execute">Application 
Configuration and Execution</a></h3>
+
+    <p>
+    Besides defining the topology, developers will also need to configure 
their applications
+    in <code>StreamsConfig</code> before running it. A complete list of
+    Kafka Streams configs can be found <a 
href="/{{version}}/documentation/#streamsconfigs"><b>here</b></a>.
+    </p>
+
+    <p>
+    Specifying the configuration in Kafka Streams is similar to the Kafka 
Producer and Consumer clients. Typically, you create a 
<code>java.util.Properties</code> instance,
+    set the necessary parameters, and construct a <code>StreamsConfig</code> 
instance from the <code>Properties</code> instance.
+    </p>
+
+    <pre>
+    import java.util.Properties;
+    import org.apache.kafka.streams.StreamsConfig;
+
+    Properties settings = new Properties();
+    // Set a few key parameters
+    settings.put(StreamsConfig.APPLICATION_ID_CONFIG, 
"my-first-streams-application");
+    settings.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "kafka-broker1:9092");
+    settings.put(StreamsConfig.ZOOKEEPER_CONNECT_CONFIG, "zookeeper1:2181");
+    // Any further settings
+    settings.put(... , ...);
+
+    // Create an instance of StreamsConfig from the Properties instance
+    StreamsConfig config = new StreamsConfig(settings);
+    </pre>
+
+    <p>
+    Apart from Kafka Streams' own configuration parameters you can also 
specify parameters for the Kafka consumers and producers that are used 
internally,
+    depending on the needs of your application. Similar to the Streams 
settings you define any such consumer and/or producer settings via 
<code>StreamsConfig</code>.
+    Note that some consumer and producer configuration parameters do use the 
same parameter name. For example, <code>send.buffer.bytes</code> or 
<code>receive.buffer.bytes</code> which
+    are used to configure TCP buffers; <code>request.timeout.ms</code> and 
<code>retry.backoff.ms</code> which control retries for client request (and 
some more).
+    If you want to set different values for consumer and producer for such a 
parameter, you can prefix the parameter name with <code>consumer.</code> or 
<code>producer.</code>:
+    </p>
+
+    <pre>
+    Properties settings = new Properties();
+    // Example of a "normal" setting for Kafka Streams
+    settings.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, 
"kafka-broker-01:9092");
+
+    // Customize the Kafka consumer settings
+    streamsSettings.put(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG, 60000);
+
+    // Customize a common client setting for both consumer and producer
+    settings.put(CommonClientConfigs.RETRY_BACKOFF_MS_CONFIG, 100L);
+
+    // Customize different values for consumer and producer
+    settings.put("consumer." + ConsumerConfig.RECEIVE_BUFFER_CONFIG, 1024 * 
1024);
+    settings.put("producer." + ProducerConfig.RECEIVE_BUFFER_CONFIG, 64 * 
1024);
+    // Alternatively, you can use
+    
settings.put(StreamsConfig.consumerPrefix(ConsumerConfig.RECEIVE_BUFFER_CONFIG),
 1024 * 1024);
+    
settings.put(StremasConfig.producerConfig(ProducerConfig.RECEIVE_BUFFER_CONFIG),
 64 * 1024);
+    </pre>
+
+    <p>
+    You can call Kafka Streams from anywhere in your application code.
+    Very commonly though you would do so within the <code>main()</code> method 
of your application, or some variant thereof.
+    </p>
+
+    <p>
+    First, you must create an instance of <code>KafkaStreams</code>. The first 
argument of the <code>KafkaStreams</code> constructor takes a topology
+    builder (either <code>KStreamBuilder</code> for the Kafka Streams DSL, or 
<code>TopologyBuilder</code> for the Processor API)
+    that is used to define a topology; The second argument is an instance of 
<code>StreamsConfig</code> mentioned above.
+    </p>
+
+    <pre>
+    import org.apache.kafka.streams.KafkaStreams;
+    import org.apache.kafka.streams.StreamsConfig;
+    import org.apache.kafka.streams.kstream.KStreamBuilder;
+    import org.apache.kafka.streams.processor.TopologyBuilder;
+
+    // Use the builders to define the actual processing topology, e.g. to 
specify
+    // from which input topics to read, which stream operations (filter, map, 
etc.)
+    // should be called, and so on.
+
+    KStreamBuilder builder = ...;  // when using the Kafka Streams DSL
+    //
+    // OR
+    //
+    TopologyBuilder builder = ...; // when using the Processor API
+
+    // Use the configuration to tell your application where the Kafka cluster 
is,
+    // which serializers/deserializers to use by default, to specify security 
settings,
+    // and so on.
+    StreamsConfig config = ...;
+
+    KafkaStreams streams = new KafkaStreams(builder, config);
+    </pre>
+
+    <p>
+    At this point, internal structures have been initialized, but the 
processing is not started yet. You have to explicitly start the Kafka Streams 
thread by calling the <code>start()</code> method:
+    </p>
+
+    <pre>
+    // Start the Kafka Streams instance
+    streams.start();
+    </pre>
+
+    <p>
+    To catch any unexpected exceptions, you may set an 
<code>java.lang.Thread.UncaughtExceptionHandler</code> before you start the 
application. This handler is called whenever a stream thread is terminated by 
an unexpected exception:
+    </p>
+
+    <pre>
+    streams.setUncaughtExceptionHandler(new Thread.UncaughtExceptionHandler() {
+    public uncaughtException(Thread t, throwable e) {
+    // here you should examine the exception and perform an appropriate action!
+    }
+    );
+    </pre>
+
+    <p>
+    To stop the application instance call the <code>close()</code> method:
+    </p>
+
+    <pre>
+    // Stop the Kafka Streams instance
+    streams.close();
+    </pre>
+
+    Now it's time to execute your application that uses the Kafka Streams 
library, which can be run just like any other Java application – there is no 
special magic or requirement on the side of Kafka Streams.
+    For example, you can package your Java application as a fat jar file and 
then start the application via:
+
+    <pre>
+    # Start the application in class `com.example.MyStreamsApp`
+    # from the fat jar named `path-to-app-fatjar.jar`.
+    $ java -cp path-to-app-fatjar.jar com.example.MyStreamsApp
+    </pre>
+
+    <p>
+    When the application instance starts running, the defined processor 
topology will be initialized as one or more stream tasks that can be executed 
in parallel by the stream threads within the instance.
+    If the processor topology defines any state stores, these state stores 
will also be (re-)constructed, if possible, during the initialization
+    period of their associated stream tasks.
+    It is important to understand that, when starting your application as 
described above, you are actually launching what Kafka Streams considers to be 
one instance of your application.
+    More than one instance of your application may be running at a time, and 
in fact the common scenario is that there are indeed multiple instances of your 
application running in parallel (e.g., on another JVM or another machine).
+    In such cases, Kafka Streams transparently re-assigns tasks from the 
existing instances to the new instance that you just started.
+    See <a href="#streams_architecture_tasks">Stream Partitions and Tasks</a> 
and <a href="#streams_architecture_threads">Threading Model</a> for details.
+    </p>
+
+    <div class="pagination">
+        <a href="/{{version}}/documentation/streams/architecture" 
class="pagination__btn pagination__btn__prev">Previous</a>
+        <a href="/{{version}}/documentation/streams/upgrade-guide" 
class="pagination__btn pagination__btn__next">Next</a>
+    </div>
+</script>
+
+<!--#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" -->
+        <ul class="breadcrumbs">
+            <li><a href="/documentation">Documentation</a></li>
+            <li><a href="/documentation/streams">Streams</a></li>
+        </ul>
+        <div class="p-content"></div>
+    </div>
+</div>
+<!--#include virtual="../../includes/_footer.htm" -->
+<script>
+$(function() {
+  // Show selected style on nav item
+  $('.b-nav__streams').addClass('selected');
+
+  // Display docs subnav items
+  $('.b-nav__docs').parent().toggleClass('nav__item__with__subs--expanded');
+});
+</script>

http://git-wip-us.apache.org/repos/asf/kafka-site/blob/04199a2e/0102/streams/index.html
----------------------------------------------------------------------
diff --git a/0102/streams/index.html b/0102/streams/index.html
new file mode 100644
index 0000000..3e21297
--- /dev/null
+++ b/0102/streams/index.html
@@ -0,0 +1,90 @@
+<!--
+ 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.
+-->
+
+<script><!--#include virtual="../js/templateData.js" --></script>
+
+<script id="streams-template" type="text/x-handlebars-template">
+    <h1>Streams</h1>
+
+    <ol class="toc">
+        <li>
+            <a href="/{{version}}/documentation/streams/core-concepts">Core 
Concepts</a>
+        </li>
+        <li>
+            <a 
href="/{{version}}/documentation/streams/architecture">Architecture</a>
+        </li>
+        <li>
+            <a 
href="/{{version}}/documentation/streams/developer-guide">Developer Guide</a>
+            <ul>
+                <li><a 
href="/{{version}}/documentation/streams/developer-guide#streams_processor">Low-level
 Processor API</a></li>
+                <li><a 
href="/{{version}}/documentation/streams/developer-guide#streams_dsl">High-level
 Streams DSL</a></li>
+                <li><a 
href="/{{version}}/documentation/streams/developer-guide#streams_execute">Application
 Configuration and Execution</a></li>
+            </ul>
+        </li>
+        <li>
+            <a href="/{{version}}/documentation/streams/upgrade-guide">Upgrade 
Guide and API Changes</a>
+        </li>
+    </ol>
+
+    <h2>Overview</h2>
+
+    <p>
+    Kafka Streams is a client library for processing and analyzing data stored 
in Kafka and either write the resulting data back to Kafka or send the final 
output to an external system. It builds upon important stream processing 
concepts such as properly distinguishing between event time and processing 
time, windowing support, and simple yet efficient management of application 
state.
+    </p>
+    <p>
+    Kafka Streams has a <b>low barrier to entry</b>: You can quickly write and 
run a small-scale proof-of-concept on a single machine; and you only need to 
run additional instances of your application on multiple machines to scale up 
to high-volume production workloads. Kafka Streams transparently handles the 
load balancing of multiple instances of the same application by leveraging 
Kafka's parallelism model.
+    </p>
+    <p>
+    Some highlights of Kafka Streams:
+    </p>
+
+    <ul>
+        <li>Designed as a <b>simple and lightweight client library</b>, which 
can be easily embedded in any Java application and integrated with any existing 
packaging, deployment and operational tools that users have for their streaming 
applications.</li>
+        <li>Has <b>no external dependencies on systems other than Apache Kafka 
itself</b> as the internal messaging layer; notably, it uses Kafka's 
partitioning model to horizontally scale processing while maintaining strong 
ordering guarantees.</li>
+        <li>Supports <b>fault-tolerant local state</b>, which enables very 
fast and efficient stateful operations like windowed joins and 
aggregations.</li>
+        <li>Employs <b>one-record-at-a-time processing</b> to achieve 
millisecond processing latency, and supports <b>event-time based windowing 
operations</b> with late arrival of records.</li>
+        <li>Offers necessary stream processing primitives, along with a 
<b>high-level Streams DSL</b> and a <b>low-level Processor API</b>.</li>
+    </ul>
+
+    <div class="pagination">
+        <a href="#" class="pagination__btn pagination__btn__prev 
pagination__btn--disabled">Previous</a>
+        <a href="/{{version}}/documentation/streams/core-concepts" 
class="pagination__btn pagination__btn__next">Next</a>
+    </div>
+</script>
+
+<!--#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" -->
+        <ul class="breadcrumbs">
+            <li><a href="/documentation">Documentation</a></li>
+        </ul>
+        <div class="p-streams"></div>
+    </div>
+</div>
+<!--#include virtual="../../includes/_footer.htm" -->
+<script>
+$(function() {
+  // Show selected style on nav item
+  $('.b-nav__streams').addClass('selected');
+
+  // Display docs subnav items
+  $('.b-nav__docs').parent().toggleClass('nav__item__with__subs--expanded');
+});
+</script>

http://git-wip-us.apache.org/repos/asf/kafka-site/blob/04199a2e/0102/streams/upgrade-guide.html
----------------------------------------------------------------------
diff --git a/0102/streams/upgrade-guide.html b/0102/streams/upgrade-guide.html
new file mode 100644
index 0000000..8eb293d
--- /dev/null
+++ b/0102/streams/upgrade-guide.html
@@ -0,0 +1,176 @@
+<!--
+ 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.
+-->
+
+<script><!--#include virtual="../js/templateData.js" --></script>
+
+<script id="content-template" type="text/x-handlebars-template">
+    <h1>Upgrade Guide &amp; API Changes</h1>
+
+    <p>
+    If you want to upgrade from 0.10.1.x to 0.10.2, see the <a 
href="/{{version}}/documentation/#upgrade_1020_streams">Upgrade Section for 
0.10.2</a>.
+    It highlights incompatible changes you need to consider to upgrade your 
code and application.
+    See <a href="#streams_api_changes_0102">below</a> a complete list of 
0.10.2 API and semantical changes that allow you to advance your application 
and/or simplify your code base, including the usage of new features.
+    </p>
+
+    <p>
+    If you want to upgrade from 0.10.0.x to 0.10.1, see the <a 
href="/{{version}}/documentation/#upgrade_1010_streams">Upgrade Section for 
0.10.1</a>.
+    It highlights incompatible changes you need to consider to upgrade your 
code and application.
+    See <a href="#streams_api_changes_0101">below</a> a complete list of 
0.10.1 API changes that allow you to advance your application and/or simplify 
your code base, including the usage of new features.
+    </p>
+
+        <h3><a id="streams_api_changes_01021" 
href="#streams_api_changes_0102">Notable changes in 0.10.2.1</a></h3>
+        <p>
+        Parameter updates in <code>StreamsConfig</code>:
+    </p>
+        <ul>
+        <li> of particular importance to improve the resiliency of a Kafka 
Streams application are two changes to default parameters of producer 
<code>retries</code> and consumer <code>max.poll.interval.ms</code> </li>
+        </ul>
+    <h3><a id="streams_api_changes_0102" 
href="#streams_api_changes_0102">Streams API changes in 0.10.2.0</a></h3>
+
+    <p>
+        New methods in <code>KafkaStreams</code>:
+    </p>
+    <ul>
+        <li> set a listener to react on application state change via 
<code>#setStateListener(StateListener listener)</code> </li>
+        <li> retrieve the current application state via <code>#state()</code> 
</li>
+        <li> retrieve the global metrics registry via <code>#metrics()</code> 
</li>
+        <li> apply a timeout when closing an application via <code>#close(long 
timeout, TimeUnit timeUnit)</code> </li>
+        <li> specify a custom indent when retrieving Kafka Streams information 
via <code>#toString(String indent)</code> </li>
+    </ul>
+
+    <p>
+        Parameter updates in <code>StreamsConfig</code>:
+    </p>
+    <ul>
+        <li> parameter <code>zookeeper.connect</code> was deprecated; a Kafka 
Streams application does no longer interact with Zookeeper for topic management 
but uses the new broker admin protocol
+            (cf. <a 
href="https://cwiki.apache.org/confluence/display/KAFKA/KIP-4+-+Command+line+and+centralized+administrative+operations#KIP-4-Commandlineandcentralizedadministrativeoperations-TopicAdminSchema.1";>KIP-4,
 Section "Topic Admin Schema"</a>) </li>
+        <li> added many new parameters for metrics, security, and client 
configurations </li>
+    </ul>
+
+    <p> Changes in <code>StreamsMetrics</code> interface: </p>
+    <ul>
+        <li> removed methods: <code>#addLatencySensor()</code> </li>
+        <li> added methods: <code>#addLatencyAndThroughputSensor()</code>, 
<code>#addThroughputSensor()</code>, <code>#recordThroughput()</code>,
+        <code>#addSensor()</code>, <code>#removeSensor()</code> </li>
+    </ul>
+
+    <p> New methods in <code>TopologyBuilder</code>: </p>
+    <ul>
+        <li> added overloads for <code>#addSource()</code> that allow to 
define a <code>auto.offset.reset</code> policy per source node </li>
+        <li> added methods <code>#addGlobalStore()</code> to add global 
<code>StateStore</code>s </li>
+    </ul>
+
+    <p> New methods in <code>KStreamBuilder</code>: </p>
+    <ul>
+        <li> added overloads for <code>#stream()</code> and 
<code>#table()</code> that allow to define a <code>auto.offset.reset</code> 
policy per input stream/table </li>
+        <li> added method <code>#globalKTable()</code> to create a 
<code>GlobalKTable</code> </li>
+    </ul>
+
+    <p> New joins for <code>KStream</code>: </p>
+    <ul>
+        <li> added overloads for <code>#join()</code> to join with 
<code>KTable</code> </li>
+        <li> added overloads for <code>#join()</code> and 
<code>leftJoin()</code> to join with <code>GlobalKTable</code> </li>
+        <li> note, join semantics in 0.10.2 were improved and thus you might 
see different result compared to 0.10.0.x and 0.10.1.x
+                (cf. <a 
href="https://cwiki.apache.org/confluence/display/KAFKA/Kafka+Streams+Join+Semantics";>Kafka
 Streams Join Semantics</a> in the Apache Kafka wiki)
+    </ul>
+
+    <p> Aligned <code>null</code>-key handling for <code>KTable</code> joins: 
</p>
+    <ul>
+        <li> like all other KTable operations, <code>KTable-KTable</code> 
joins do not throw an exception on <code>null</code> key records anymore, but 
drop those records silently </li>
+    </ul>
+
+    <p> New window type <em>Session Windows</em>: </p>
+    <ul>
+        <li> added class <code>SessionWindows</code> to specify session 
windows </li>
+        <li> added overloads for <code>KGroupedStream</code> methods 
<code>#count()</code>, <code>#reduce()</code>, and <code>#aggregate()</code>
+                to allow session window aggregations </li>
+    </ul>
+
+    <p> Changes to <code>TimestampExtractor</code>: </p>
+    <ul>
+        <li> method <code>#extract()</code> has a second parameter now </li>
+        <li> new default timestamp extractor class 
<code>FailOnInvalidTimestamp</code>
+                (it gives the same behavior as old (and removed) default 
extractor <code>ConsumerRecordTimestampExtractor</code>) </li>
+        <li> new alternative timestamp extractor classes 
<code>LogAndSkipOnInvalidTimestamp</code> and 
<code>UsePreviousTimeOnInvalidTimestamps</code> </li>
+    </ul>
+
+    <p> Relaxed type constraints of many DSL interfaces, classes, and methods 
(cf. <a 
href="https://cwiki.apache.org/confluence/display/KAFKA/KIP-100+-+Relax+Type+constraints+in+Kafka+Streams+API";>KIP-100</a>).
 </p>
+
+    <h3><a id="streams_api_changes_0101" 
href="#streams_api_changes_0101">Streams API changes in 0.10.1.0</a></h3>
+
+    <p> Stream grouping and aggregation split into two methods: </p>
+    <ul>
+        <li> old: KStream #aggregateByKey(), #reduceByKey(), and #countByKey() 
</li>
+        <li> new: KStream#groupByKey() plus KGroupedStream #aggregate(), 
#reduce(), and #count() </li>
+        <li> Example: stream.countByKey() changes to 
stream.groupByKey().count() </li>
+    </ul>
+
+    <p> Auto Repartitioning: </p>
+    <ul>
+        <li> a call to through() after a key-changing operator and before an 
aggregation/join is no longer required </li>
+        <li> Example: stream.selectKey(...).through(...).countByKey() changes 
to stream.selectKey().groupByKey().count() </li>
+    </ul>
+
+    <p> TopologyBuilder: </p>
+    <ul>
+        <li> methods #sourceTopics(String applicationId) and 
#topicGroups(String applicationId) got simplified to #sourceTopics() and 
#topicGroups() </li>
+    </ul>
+
+    <p> DSL: new parameter to specify state store names: </p>
+    <ul>
+        <li> The new Interactive Queries feature requires to specify a store 
name for all source KTables and window aggregation result KTables (previous 
parameter "operator/window name" is now the storeName) </li>
+        <li> KStreamBuilder#table(String topic) changes to #topic(String 
topic, String storeName) </li>
+        <li> KTable#through(String topic) changes to #through(String topic, 
String storeName) </li>
+        <li> KGroupedStream #aggregate(), #reduce(), and #count() require 
additional parameter "String storeName"</li>
+        <li> Example: stream.countByKey(TimeWindows.of("windowName", 1000)) 
changes to stream.groupByKey().count(TimeWindows.of(1000), "countStoreName") 
</li>
+    </ul>
+
+    <p> Windowing: </p>
+    <ul>
+        <li> Windows are not named anymore: TimeWindows.of("name", 1000) 
changes to TimeWindows.of(1000) (cf. DSL: new parameter to specify state store 
names) </li>
+        <li> JoinWindows has no default size anymore: 
JoinWindows.of("name").within(1000) changes to JoinWindows.of(1000) </li>
+    </ul>
+
+    <div class="pagination">
+        <a href="/{{version}}/documentation/streams/developer-guide" 
class="pagination__btn pagination__btn__prev">Previous</a>
+        <a href="#" class="pagination__btn pagination__btn__next 
pagination__btn--disabled">Next</a>
+    </div>
+</script>
+
+<!--#include virtual="../../includes/_header.htm" -->
+<!--#include virtual="../../includes/_top.htm" -->
+<div class="content documentation">
+       <!--#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>
+        </ul>
+        <div class="p-content"></div>
+    </div>
+</div>
+<!--#include virtual="../../includes/_footer.htm" -->
+<script>
+$(function() {
+  // Show selected style on nav item
+  $('.b-nav__streams').addClass('selected');
+
+  // Display docs subnav items
+  $('.b-nav__docs').parent().toggleClass('nav__item__with__subs--expanded');
+});
+</script>

http://git-wip-us.apache.org/repos/asf/kafka-site/blob/04199a2e/0110/documentation/index.html
----------------------------------------------------------------------
diff --git a/0110/documentation/index.html b/0110/documentation/index.html
new file mode 100644
index 0000000..698eeed
--- /dev/null
+++ b/0110/documentation/index.html
@@ -0,0 +1 @@
+<!--#include virtual="../documentation.html" -->
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/kafka-site/blob/04199a2e/0110/documentation/streams.html
----------------------------------------------------------------------
diff --git a/0110/documentation/streams.html b/0110/documentation/streams.html
deleted file mode 100644
index d8d2bb2..0000000
--- a/0110/documentation/streams.html
+++ /dev/null
@@ -1,19 +0,0 @@
-<!--
- 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.
--->
-
-<!-- should always link the the latest release's documentation -->
-<!--#include virtual="../streams.html" -->

http://git-wip-us.apache.org/repos/asf/kafka-site/blob/04199a2e/0110/documentation/streams/architecture.html
----------------------------------------------------------------------
diff --git a/0110/documentation/streams/architecture.html 
b/0110/documentation/streams/architecture.html
new file mode 100644
index 0000000..6ba69f3
--- /dev/null
+++ b/0110/documentation/streams/architecture.html
@@ -0,0 +1,19 @@
+<!--
+ 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.
+-->
+
+<!-- should always link the the latest release's documentation -->
+<!--#include virtual="../../streams/architecture.html" -->

http://git-wip-us.apache.org/repos/asf/kafka-site/blob/04199a2e/0110/documentation/streams/core-concepts.html
----------------------------------------------------------------------
diff --git a/0110/documentation/streams/core-concepts.html 
b/0110/documentation/streams/core-concepts.html
new file mode 100644
index 0000000..ff46c53
--- /dev/null
+++ b/0110/documentation/streams/core-concepts.html
@@ -0,0 +1,19 @@
+<!--
+ 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.
+-->
+
+<!-- should always link the the latest release's documentation -->
+<!--#include virtual="../../streams/core-concepts.html" -->

http://git-wip-us.apache.org/repos/asf/kafka-site/blob/04199a2e/0110/documentation/streams/developer-guide.html
----------------------------------------------------------------------
diff --git a/0110/documentation/streams/developer-guide.html 
b/0110/documentation/streams/developer-guide.html
new file mode 100644
index 0000000..e258331
--- /dev/null
+++ b/0110/documentation/streams/developer-guide.html
@@ -0,0 +1,19 @@
+<!--
+ 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.
+-->
+
+<!-- should always link the the latest release's documentation -->
+<!--#include virtual="../../streams/developer-guide.html" -->

http://git-wip-us.apache.org/repos/asf/kafka-site/blob/04199a2e/0110/documentation/streams/index.html
----------------------------------------------------------------------
diff --git a/0110/documentation/streams/index.html 
b/0110/documentation/streams/index.html
new file mode 100644
index 0000000..1aaaff4
--- /dev/null
+++ b/0110/documentation/streams/index.html
@@ -0,0 +1,19 @@
+<!--
+ 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.
+-->
+
+<!-- should always link the the latest release's documentation -->
+<!--#include virtual="../../streams/index.html" -->

http://git-wip-us.apache.org/repos/asf/kafka-site/blob/04199a2e/0110/documentation/streams/upgrade-guide.html
----------------------------------------------------------------------
diff --git a/0110/documentation/streams/upgrade-guide.html 
b/0110/documentation/streams/upgrade-guide.html
new file mode 100644
index 0000000..0c68795
--- /dev/null
+++ b/0110/documentation/streams/upgrade-guide.html
@@ -0,0 +1,19 @@
+<!--
+ 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.
+-->
+
+<!-- should always link the the latest release's documentation -->
+<!--#include virtual="../../streams/upgrade-guide.html" -->

Reply via email to