kafka-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From guozh...@apache.org
Subject kafka git commit: HOTFIX: fixed section incompatible Steams API changes
Date Wed, 15 Feb 2017 21:17:32 GMT
Repository: kafka
Updated Branches:
  refs/heads/trunk 4db048d61 -> 2c91b324d


HOTFIX: fixed section incompatible Steams API changes

Author: Matthias J. Sax <matthias@confluent.io>

Reviewers: Damian Guy, Guozhang Wang

Closes #2492 from mjsax/hotfixDocs


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

Branch: refs/heads/trunk
Commit: 2c91b324d4798138dd479f54269bdc4f39339817
Parents: 4db048d
Author: Matthias J. Sax <matthias@confluent.io>
Authored: Wed Feb 15 13:17:29 2017 -0800
Committer: Guozhang Wang <wangguoz@gmail.com>
Committed: Wed Feb 15 13:17:29 2017 -0800

----------------------------------------------------------------------
 docs/api.html          |   2 +-
 docs/introduction.html |   6 +--
 docs/streams.html      | 123 ++++++++++++++++++++++++++++++--------------
 docs/toc.html          |   4 +-
 docs/upgrade.html      |  47 +++--------------
 docs/uses.html         |  49 ++++++++++++++----
 6 files changed, 137 insertions(+), 94 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kafka/blob/2c91b324/docs/api.html
----------------------------------------------------------------------
diff --git a/docs/api.html b/docs/api.html
index 9b9cd96..de0bb1d 100644
--- a/docs/api.html
+++ b/docs/api.html
@@ -66,7 +66,7 @@
 	Examples showing how to use this library are given in the
 	<a href="/{{version}}/javadoc/index.html?org/apache/kafka/streams/KafkaStreams.html"
title="Kafka 0.10.2 Javadoc">javadocs</a>
 	<p>
-	Additional documentation on using the Streams API is available <a href="/documentation.html#streams">here</a>.
+	Additional documentation on using the Streams API is available <a href="/{{version}}/documentation/streams">here</a>.
 	<p>
 	To use Kafka Streams you can use the following maven dependency:
 

http://git-wip-us.apache.org/repos/asf/kafka/blob/2c91b324/docs/introduction.html
----------------------------------------------------------------------
diff --git a/docs/introduction.html b/docs/introduction.html
index 7672a51..556aa02 100644
--- a/docs/introduction.html
+++ b/docs/introduction.html
@@ -43,7 +43,7 @@
       <ul style="float: left; width: 40%;">
       <li>The <a href="/documentation.html#producerapi">Producer API</a>
allows an application to publish a stream of records to one or more Kafka topics.
       <li>The <a href="/documentation.html#consumerapi">Consumer API</a>
allows an application to subscribe to one or more topics and process the stream of records
produced to them.
-    <li>The <a href="/documentation.html#streams">Streams API</a> allows
an application to act as a <i>stream processor</i>, consuming an input stream
from one or more topics and producing an output stream to one or more output topics, effectively
transforming the input streams to output streams.
+    <li>The <a href="/documentation/streams">Streams API</a> allows an
application to act as a <i>stream processor</i>, consuming an input stream from
one or more topics and producing an output stream to one or more output topics, effectively
transforming the input streams to output streams.
     <li>The <a href="/documentation.html#connect">Connector API</a> allows
building and running reusable producers or consumers that connect Kafka topics to existing
applications or data systems. For example, a connector to a relational database might capture
every change to a table.
   </ul>
       <img src="/{{version}}/images/kafka-apis.png" style="float: right; width: 50%;">
@@ -171,7 +171,7 @@
   For example, a retail application might take in input streams of sales and shipments, and
output a stream of reorders and price adjustments computed off this data.
   </p>
   <p>
-  It is possible to do simple processing directly using the producer and consumer APIs. However
for more complex transformations Kafka provides a fully integrated <a href="/documentation.html#streams">Streams
API</a>. This allows building applications that do non-trivial processing that compute
aggregations off of streams or join streams together.
+  It is possible to do simple processing directly using the producer and consumer APIs. However
for more complex transformations Kafka provides a fully integrated <a href="/documentation/streams">Streams
API</a>. This allows building applications that do non-trivial processing that compute
aggregations off of streams or join streams together.
   </p>
   <p>
   This facility helps solve the hard problems this type of application faces: handling out-of-order
data, reprocessing input as code changes, performing stateful computations, etc.
@@ -203,4 +203,4 @@
   </p>
 </script>
 
-<div class="p-introduction"></div>
\ No newline at end of file
+<div class="p-introduction"></div>

http://git-wip-us.apache.org/repos/asf/kafka/blob/2c91b324/docs/streams.html
----------------------------------------------------------------------
diff --git a/docs/streams.html b/docs/streams.html
index 94ce7a9..31bae5d 100644
--- a/docs/streams.html
+++ b/docs/streams.html
@@ -39,7 +39,7 @@
                 </ul>
             </li>
             <li>
-                <a href="#streams_upgrade">Upgrade Guide and API Changes</a>
+                <a href="#streams_upgrade_and_api">Upgrade Guide and API Changes</a>
             </li>
         </ol>
 
@@ -230,7 +230,7 @@
 
         <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
+        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>
 
@@ -257,7 +257,7 @@
         <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="/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.
+        <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>
@@ -266,14 +266,14 @@
         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="/documentation/#streamsconfigs">Kafka Streams Configs</a> Section.
+        the task (re)initialization cost. See <code>num.standby.replicas</code>
at the <a href="/{{version}}/documentation/#streamsconfigs">Kafka Streams Configs</a>
Section.
         </p>
         <br>
 
         <h2><a id="streams_developer" href="#streams_developer">Developer Guide</a></h2>
 
         <p>
-        There is a <a href="/documentation/#quickstart_kafkastreams">quickstart</a>
example that provides how to run a stream processing program coded in the Kafka Streams library.
+        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>
 
@@ -470,7 +470,7 @@
 
         <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>.
+        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>
 
@@ -567,7 +567,8 @@
             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 semantics
are similar to the corresponding operators in relational databases.
+        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.
@@ -654,7 +655,7 @@
         <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="/documentation/#streamsconfigs"><b>here</b></a>.
+        Kafka Streams configs can be found <a href="/{{version}}/documentation/#streamsconfigs"><b>here</b></a>.
         </p>
 
         <p>
@@ -790,13 +791,21 @@
         </p>
         <br>
 
-        <h2><a id="streams_upgrade" href="#streams_upgrade">Upgrade Guide and
API Changes</a></h2>
+        <h2><a id="streams_upgrade_and_api" href="#streams_upgrade_and_api">Upgrade
Guide and API Changes</a></h2>
 
         <p>
-        See the <a href="/documentation/#upgrade_1020_streams">Upgrade Section</a>
for upgrading a Kafka Streams Application from 0.10.1.x to 0.10.2.0.
+        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>
 
-        <h3><a id="streams_api_changes" href="#streams_api_changes">Streams API
changes in 0.10.2.0</a></h3>
+        <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_0102" href="#streams_api_changes_0102">Streams
API changes in 0.10.2.0</a></h3>
 
         <p>
             New methods in <code>KafkaStreams</code>:
@@ -824,46 +833,84 @@
             <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>
+        <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> <code>#table()</code> always requires store name </li>
-                <li> added method <code>#globalKTable()</code> to create
a <code>GlobalKTable</code> </li>
-            </ul>
+        <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>
-            </ul>
+        <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>
+        <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>
+        <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>
+        <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>
+
 </script>
 
 <!--#include virtual="../includes/_header.htm" -->

http://git-wip-us.apache.org/repos/asf/kafka/blob/2c91b324/docs/toc.html
----------------------------------------------------------------------
diff --git a/docs/toc.html b/docs/toc.html
index 792dc4e..787153d 100644
--- a/docs/toc.html
+++ b/docs/toc.html
@@ -144,11 +144,11 @@
                     <li><a href="/{{version}}/documentation/streams#streams_dsl">High-Level
Streams DSL</a></li>
                     <li><a href="/{{version}}/documentation/streams#streams_execute">Application
Configuration and Execution</a></li>
                 </ul>
-                <li><a href="/{{version}}/documentation/streams#streams_upgrade">9.5
Upgrade Guide and API Changes</a></li>
+                <li><a href="/{{version}}/documentation/streams#streams_upgrade_and_api">9.5
Upgrade Guide and API Changes</a></li>
             </ul>
         </li>
     </ul>
 
 </script>
 
-<div class="p-toc"></div>
\ No newline at end of file
+<div class="p-toc"></div>

http://git-wip-us.apache.org/repos/asf/kafka/blob/2c91b324/docs/upgrade.html
----------------------------------------------------------------------
diff --git a/docs/upgrade.html b/docs/upgrade.html
index 3613734..5976054 100644
--- a/docs/upgrade.html
+++ b/docs/upgrade.html
@@ -47,15 +47,14 @@ Kafka cluster before upgrading your clients. Version 0.10.2 brokers support
0.8.
 
 <p><b>Note:</b> Bumping the protocol version and restarting can be done
any time after the brokers were upgraded. It does not have to be immediately after.
 
-<h5><a id="upgrade_1020_streams" href="#upgrade_1020_streams">Upgrading a Kafka
Streams Application</a></h5>
+<h5><a id="upgrade_1020_streams" href="#upgrade_1020_streams">Upgrading a 0.10.1
Kafka Streams Application</a></h5>
 <ul>
     <li> Upgrading your Streams application from 0.10.1 to 0.10.2 does not require
a broker upgrade.
          A Kafka Streams 0.10.2 application can connect to 0.10.2 and 0.10.1 brokers (it
is not possible to connect to 0.10.0 brokers though). </li>
     <li> You need to recompile your code. Just swapping the Kafka Streams library jar
file will not work and will break your application. </li>
-    <li> <code>KStreamBuilder#table()</code> always requires a store name.
</li>
-    <li> <code>KTable#through()</code> always requires a store name. </li>
     <li> If you use a custom (i.e., user implemented) timestamp extractor, you will
need to update this code, because the <code>TimestampExtractor</code> interface
was changed. </li>
     <li> If you register custom metrics, you will need to update this code, because
the <code>StreamsMetric</code> interface was changed. </li>
+    <li> See <a href="/{{version}}/documentation/streams#streams_api_changes_0102">Streams
API changes in 0.10.2</a> for more details. </li>
 </ul>
 
 <h5><a id="upgrade_1020_notable" href="#upgrade_1020_notable">Notable changes
in 0.10.2.0</a></h5>
@@ -75,7 +74,7 @@ Kafka cluster before upgrading your clients. Version 0.10.2 brokers support
0.8.
         modifying Zookeeper directly. This eliminates the need for privileges to access Zookeeper
directly and "StreamsConfig.ZOOKEEPER_CONFIG"
         should not be set in the Streams app any more. If the Kafka cluster is secured, Streams
apps must have the required security privileges to create new topics.</li>
     <li>Several new fields including "security.protocol", "connections.max.idle.ms",
"retry.backoff.ms", "reconnect.backoff.ms" and "request.timeout.ms" were added to
-        StreamsConfig class. User should pay attenntion to the default values and set these
if needed. For more details please refer to <a href="#streamsconfigs">3.5 Kafka Streams
Configs</a>.</li>
+        StreamsConfig class. User should pay attention to the default values and set these
if needed. For more details please refer to <a href="/{{version}}/documentation/#streamsconfigs">3.5
Kafka Streams Configs</a>.</li>
     <li>The <code>offsets.topic.replication.factor</code> broker config
is now enforced upon auto topic creation. Internal auto topic creation will fail with a GROUP_COORDINATOR_NOT_AVAILABLE
error until the cluster size meets this replication factor requirement.</li>
 </ul>
 
@@ -126,41 +125,11 @@ only support 0.10.1.x or later brokers while 0.10.1.x brokers also support
older
     <li> Due to the increased number of index files, on some brokers with large amount
the log segments (e.g. >15K), the log loading process during the broker startup could be
longer. Based on our experiment, setting the num.recovery.threads.per.data.dir to one may
reduce the log loading time. </li>
 </ul>
 
-<h5><a id="upgrade_1010_streams" href="#upgrade_1010_streams">Streams API changes
in 0.10.1.0</a></h5>
+<h5><a id="upgrade_1010_streams" href="#upgrade_1010_streams">Upgrading a 0.10.0
Kafka Streams Application</a></h5>
 <ul>
-    <li> Stream grouping and aggregation split into two methods:
-        <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>
-    </li>
-    <li> Auto Repartitioning:
-        <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>
-    </li>
-    <li> TopologyBuilder:
-        <ul>
-            <li> methods #sourceTopics(String applicationId) and #topicGroups(String
applicationId) got simplified to #sourceTopics() and #topicGroups() </li>
-        </ul>
-    </li>
-    <li> DSL: new parameter to specify state store names:
-        <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>
-    </li>
-    <li> Windowing:
-        <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>
-    </li>
+    <li> Upgrading your Streams application from 0.10.0 to 0.10.1 does require a <a
href="#upgrade_10_1">broker upgrade</a> because a Kafka Streams 0.10.1 application
can only connect to 0.10.1 brokers. </li>
+    <li> There are couple of API changes, that are not backward compatible (cf. <a
href="/{{version}}/documentation/streams#streams_api_changes_0101">Streams API changes
in 0.10.1</a> for more details).
+         Thus, you need to update and recompile your code. Just swapping the Kafka Streams
library jar file will not work and will break your application. </li>
 </ul>
 
 <h5><a id="upgrade_1010_notable" href="#upgrade_1010_notable">Notable changes
in 0.10.1.0</a></h5>
@@ -300,7 +269,7 @@ work with 0.10.0.x brokers. Therefore, 0.9.0.0 clients should be upgraded
to 0.9
 <h5><a id="upgrade_10_notable" href="#upgrade_10_notable">Notable changes in
0.10.0.0</a></h5>
 
 <ul>
-    <li> Starting from Kafka 0.10.0.0, a new client library named <b>Kafka Streams</b>
is available for stream processing on data stored in Kafka topics. This new client library
only works with 0.10.x and upward versioned brokers due to message format changes mentioned
above. For more information please read <a href="#streams_overview">this section</a>.</li>
+    <li> Starting from Kafka 0.10.0.0, a new client library named <b>Kafka Streams</b>
is available for stream processing on data stored in Kafka topics. This new client library
only works with 0.10.x and upward versioned brokers due to message format changes mentioned
above. For more information please read <a href="/{{version}}/documentation/stream">Streams
documentation</a>.</li>
     <li> The default value of the configuration parameter <code>receive.buffer.bytes</code>
is now 64K for the new consumer.</li>
     <li> The new consumer now exposes the configuration parameter <code>exclude.internal.topics</code>
to restrict internal topics (such as the consumer offsets topic) from accidentally being included
in regular expression subscriptions. By default, it is enabled.</li>
     <li> The old Scala producer has been deprecated. Users should migrate their code
to the Java producer included in the kafka-clients JAR as soon as possible. </li>

http://git-wip-us.apache.org/repos/asf/kafka/blob/2c91b324/docs/uses.html
----------------------------------------------------------------------
diff --git a/docs/uses.html b/docs/uses.html
index 2d238c2..4e88859 100644
--- a/docs/uses.html
+++ b/docs/uses.html
@@ -15,40 +15,67 @@
  limitations under the License.
 -->
 
-<p> Here is a description of a few of the popular use cases for Apache Kafka&trade;.
For an overview of a number of these areas in action, see <a href="http://engineering.linkedin.com/distributed-systems/log-what-every-software-engineer-should-know-about-real-time-datas-unifying">this
blog post</a>. </p>
+<p> Here is a description of a few of the popular use cases for Apache Kafka&trade;.
+For an overview of a number of these areas in action, see <a href="https://engineering.linkedin.com/distributed-systems/log-what-every-software-engineer-should-know-about-real-time-datas-unifying/">this
blog post</a>. </p>
 
 <h4><a id="uses_messaging" href="#uses_messaging">Messaging</a></h4>
 
-Kafka works well as a replacement for a more traditional message broker. Message brokers
are used for a variety of reasons (to decouple processing from data producers, to buffer unprocessed
messages, etc). In comparison to most messaging systems Kafka has better throughput, built-in
partitioning, replication, and fault-tolerance which makes it a good solution for large scale
message processing applications.
+Kafka works well as a replacement for a more traditional message broker.
+Message brokers are used for a variety of reasons (to decouple processing from data producers,
to buffer unprocessed messages, etc).
+In comparison to most messaging systems Kafka has better throughput, built-in partitioning,
replication, and fault-tolerance which makes it a good
+solution for large scale message processing applications.
 <p>
-In our experience messaging uses are often comparatively low-throughput, but may require
low end-to-end latency and often depend on the strong durability guarantees Kafka provides.
+In our experience messaging uses are often comparatively low-throughput, but may require
low end-to-end latency and often depend on the strong
+durability guarantees Kafka provides.
 <p>
-In this domain Kafka is comparable to traditional messaging systems such as <a href="http://activemq.apache.org">ActiveMQ</a>
or <a href="https://www.rabbitmq.com">RabbitMQ</a>.
+In this domain Kafka is comparable to traditional messaging systems such as <a href="http://activemq.apache.org">ActiveMQ</a>
or
+<a href="https://www.rabbitmq.com">RabbitMQ</a>.
 
 <h4><a id="uses_website" href="#uses_website">Website Activity Tracking</a></h4>
 
-The original use case for Kafka was to be able to rebuild a user activity tracking pipeline
as a set of real-time publish-subscribe feeds. This means site activity (page views, searches,
or other actions users may take) is published to central topics with one topic per activity
type. These feeds are available for subscription for a range of use cases including real-time
processing, real-time monitoring, and loading into Hadoop or offline data warehousing systems
for offline processing and reporting.
+The original use case for Kafka was to be able to rebuild a user activity tracking pipeline
as a set of real-time publish-subscribe feeds.
+This means site activity (page views, searches, or other actions users may take) is published
to central topics with one topic per activity type.
+These feeds are available for subscription for a range of use cases including real-time processing,
real-time monitoring, and loading into Hadoop or
+offline data warehousing systems for offline processing and reporting.
 <p>
 Activity tracking is often very high volume as many activity messages are generated for each
user page view.
 
 <h4><a id="uses_metrics" href="#uses_metrics">Metrics</a></h4>
 
-Kafka is often used for operational monitoring data. This involves aggregating statistics
from distributed applications to produce centralized feeds of operational data.
+Kafka is often used for operational monitoring data.
+This involves aggregating statistics from distributed applications to produce centralized
feeds of operational data.
 
 <h4><a id="uses_logs" href="#uses_logs">Log Aggregation</a></h4>
 
-Many people use Kafka as a replacement for a log aggregation solution. Log aggregation typically
collects physical log files off servers and puts them in a central place (a file server or
HDFS perhaps) for processing. Kafka abstracts away the details of files and gives a cleaner
abstraction of log or event data as a stream of messages. This allows for lower-latency processing
and easier support for multiple data sources and distributed data consumption.
+Many people use Kafka as a replacement for a log aggregation solution.
+Log aggregation typically collects physical log files off servers and puts them in a central
place (a file server or HDFS perhaps) for processing.
+Kafka abstracts away the details of files and gives a cleaner abstraction of log or event
data as a stream of messages.
+This allows for lower-latency processing and easier support for multiple data sources and
distributed data consumption.
 
-In comparison to log-centric systems like Scribe or Flume, Kafka offers equally good performance,
stronger durability guarantees due to replication, and much lower end-to-end latency.
+In comparison to log-centric systems like Scribe or Flume, Kafka offers equally good performance,
stronger durability guarantees due to replication,
+and much lower end-to-end latency.
 
 <h4><a id="uses_streamprocessing" href="#uses_streamprocessing">Stream Processing</a></h4>
 
-Many users of Kafka process data in processing pipelines consisting of multiple stages, where
raw input data is consumed from Kafka topics and then aggregated, enriched, or otherwise transformed
into new topics for further consumption or follow-up processing. For example, a processing
pipeline for recommending news articles might crawl article content from RSS feeds and publish
it to an "articles" topic; further processing might normalize or deduplicate this content
and published the cleansed article content to a new topic; a final processing stage might
attempt to recommend this content to users. Such processing pipelines create graphs of real-time
data flows based on the individual topics. Starting in 0.10.0.0, a light-weight but powerful
stream processing library called <a href="#streams_overview">Kafka Streams</a>
is available in Apache Kafka to perform such data processing as described above. Apart from
Kafka Streams, alternative open source stream processing tools include <a h
 ref="https://storm.apache.org/">Apache Storm</a> and <a href="http://samza.apache.org/">Apache
Samza</a>.
+Many users of Kafka process data in processing pipelines consisting of multiple stages, where
raw input data is consumed from Kafka topics and then
+aggregated, enriched, or otherwise transformed into new topics for further consumption or
follow-up processing.
+For example, a processing pipeline for recommending news articles might crawl article content
from RSS feeds and publish it to an "articles" topic;
+further processing might normalize or deduplicate this content and published the cleansed
article content to a new topic;
+a final processing stage might attempt to recommend this content to users.
+Such processing pipelines create graphs of real-time data flows based on the individual topics.
+Starting in 0.10.0.0, a light-weight but powerful stream processing library called <a
href="/{{version}}/documentation/streams">Kafka Streams</a>
+is available in Apache Kafka to perform such data processing as described above.
+Apart from Kafka Streams, alternative open source stream processing tools include <a href="https://storm.apache.org/">Apache
Storm</a> and
+<a href="http://samza.apache.org/">Apache Samza</a>.
 
 <h4><a id="uses_eventsourcing" href="#uses_eventsourcing">Event Sourcing</a></h4>
 
-<a href="http://martinfowler.com/eaaDev/EventSourcing.html">Event sourcing</a>
is a style of application design where state changes are logged as a time-ordered sequence
of records. Kafka's support for very large stored log data makes it an excellent backend for
an application built in this style.
+<a href="http://martinfowler.com/eaaDev/EventSourcing.html">Event sourcing</a>
is a style of application design where state changes are logged as a
+time-ordered sequence of records. Kafka's support for very large stored log data makes it
an excellent backend for an application built in this style.
 
 <h4><a id="uses_commitlog" href="#uses_commitlog">Commit Log</a></h4>
 
-Kafka can serve as a kind of external commit-log for a distributed system. The log helps
replicate data between nodes and acts as a re-syncing mechanism for failed nodes to restore
their data. The <a href="/documentation.html#compaction">log compaction</a> feature
in Kafka helps support this usage. In this usage Kafka is similar to <a href="http://zookeeper.apache.org/bookkeeper/">Apache
BookKeeper</a> project.
+Kafka can serve as a kind of external commit-log for a distributed system. The log helps
replicate data between nodes and acts as a re-syncing
+mechanism for failed nodes to restore their data.
+The <a href="/documentation.html#compaction">log compaction</a> feature in Kafka
helps support this usage.
+In this usage Kafka is similar to <a href="http://zookeeper.apache.org/bookkeeper/">Apache
BookKeeper</a> project.


Mime
View raw message