KAFKA-5823: Extend upgrade section for KIP-120

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

Reviewers: Sharon Liu <sharonliu.cup@gmail.com>, Damian Guy <damian.guy@gmail.com>, Guozhang Wang <wangguoz@gmail.com>

Closes #3787 from mjsax/kafka-5823-kip120-docs
This commit is contained in:
Matthias J. Sax 2017-09-05 16:54:00 -07:00 committed by Guozhang Wang
parent 5d69966195
commit cd59976eea
1 changed files with 39 additions and 1 deletions

View File

@ -45,7 +45,45 @@
</p>
<h3><a id="streams_api_changes_100" href="#streams_api_changes_100">Streams API changes in 1.0.0</a></h3>
<!-- TODO: KIP-120, 161, 167, 173 -->
<!-- TODO: 161, 167, 173, 182 -->
<p>
With 1.0 a major API refactoring was accomplished and the new API is cleaner and easier to use.
This change includes the five main classes <code>KafakStreams<code>, <code>KStreamBuilder</code>,
<code>KStream</code>, <code>KTable</code>, and <code>TopologyBuilder</code> (and some more others).
All changes are fully backward compatible as old API is only deprecated but not removed.
We recommend to move to the new API as soon as you can.
We will summarize all API changes in the next paragraphs.
</p>
<p>
The two main classes to specify a topology via the DSL (<code>KStreamBuilder</code>)
or the Processor API (<code>TopologyBuilder</code>) were deprecated and replaced by
<code>StreamsBuilder</code> and <code>Topology<code> (both new classes are located in
package <code>org.apache.kafka.streams</code>).
Note, that <code>StreamsBuilder</code> does not extend <code>Topology</code>, i.e.,
the class hierarchy is different now.
The new classes have basically the same methods as the old ones to build a topology via DSL or Processor API.
However, some internal methods that were public in <code>KStreamBuilder</code>
and <code>TopologyBuilder</code> but not part of the actual API are not present
in the new classe any longer.
Furthermore, some overloads were simplified compared to the original classes.
See <a href="https://cwiki.apache.org/confluence/display/KAFKA/KIP-120%3A+Cleanup+Kafka+Streams+builder+API">KIP-120</a>
and <a href="https://cwiki.apache.org/confluence/display/KAFKA/KIP-182%3A+Reduce+Streams+DSL+overloads+and+allow+easier+use+of+custom+storage+engines">KIP-182</a>
for full details.
</p>
<p>
Changing how a topology is specified also affects <code>KafkaStreams<code> constructors,
that now only accept a <code>Topology</code>.
Using the DSL builder class <code>StreamsBuilder</code> one can get the constructed
<code>Topology</code> via <code>StreamsBuilder#build()</code>.
Additionally, a new class <code>org.apache.kafka.streams.TopologyDescription</code>
(and some more dependent classes) were added.
Those can be used to get a detailed description of the specified topology
and can be obtained by calling <code>Topology#describe()</code>.
An example using this new API is shown in the <a href="/{{version}}/documentation/streams/quickstart">quickstart section</a>.
</p>
<p>
The Processor API was extended to allow users to schedule <code>punctuate</code> functions either based on data-driven <b>stream time</b> or wall-clock time.