From cd59976eea54efe6367b4c55872ab206f504829a Mon Sep 17 00:00:00 2001
From: "Matthias J. Sax"
+ 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 KafakStreams, KStreamBuilder,
+ KStream, KTable, and TopologyBuilder (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.
+
+ The two main classes to specify a topology via the DSL (KStreamBuilder)
+ or the Processor API (TopologyBuilder) were deprecated and replaced by
+ StreamsBuilder and Topology (both new classes are located in
+ package org.apache.kafka.streams).
+ Note, that StreamsBuilder does not extend Topology, 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 KStreamBuilder
+ and TopologyBuilder 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 KIP-120
+ and KIP-182
+ for full details.
+
+ Changing how a topology is specified also affects KafkaStreams constructors,
+ that now only accept a Topology.
+ Using the DSL builder class StreamsBuilder one can get the constructed
+ Topology via StreamsBuilder#build().
+ Additionally, a new class org.apache.kafka.streams.TopologyDescription
+ (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 Topology#describe().
+ An example using this new API is shown in the quickstart section.
+
The Processor API was extended to allow users to schedule punctuate functions either based on data-driven stream time or wall-clock time.