mirror of https://github.com/apache/kafka.git
MINOR: Add Kafka Streams API / upgrade notes
Author: Guozhang Wang <wangguoz@gmail.com> Reviewers: Michael G. Noll <michael@confluent.io>, Ismael Juma <ismael@juma.me.uk> Closes #1321 from guozhangwang/KStreamsJavaDoc
This commit is contained in:
parent
9575e93070
commit
6f1873242c
|
@ -165,3 +165,22 @@ This new unified consumer API removes the distinction between the 0.8 high-level
|
|||
|
||||
Examples showing how to use the consumer are given in the
|
||||
<a href="http://kafka.apache.org/0100/javadoc/index.html?org/apache/kafka/clients/consumer/KafkaConsumer.html" title="Kafka 0.9.0 Javadoc">javadocs</a>.
|
||||
|
||||
<h3><a id="streamsapi" href="#streamsapi">2.3 Streams API</a></h3>
|
||||
|
||||
As of the 0.10.0 release we have added a new client library named <b>Kafka Streams</b> to let users implement their stream processing
|
||||
applications with data stored in Kafka topics. Kafka Streams is considered alpha quality and its public APIs are likely to change in
|
||||
future releases.
|
||||
You can use Kafka Streams by adding a dependency on the streams jar using
|
||||
the following example maven co-ordinates (you can change the version numbers with new releases):
|
||||
|
||||
<pre>
|
||||
<dependency>
|
||||
<groupId>org.apache.kafka</groupId>
|
||||
<artifactId>kafka-streams</artifactId>
|
||||
<version>0.10.0.0</version>
|
||||
</dependency>
|
||||
</pre>
|
||||
|
||||
Examples showing how to use this library are given in the
|
||||
<a href="http://kafka.apache.org/0100/javadoc/index.html?org/apache/kafka/streams/KafkaStreams.html" title="Kafka 0.10.0 Javadoc">javadocs</a> (note those classes annotated with <b>@InterfaceStability.Unstable</b>, indicating their public APIs may change without backward-compatibility in future releases).
|
|
@ -40,6 +40,7 @@ Prior releases: <a href="/07/documentation.html">0.7.x</a>, <a href="/08/documen
|
|||
<li><a href="#simpleconsumerapi">2.2.2 Old Simple Consumer API</a>
|
||||
<li><a href="#newconsumerapi">2.2.3 New Consumer API</a>
|
||||
</ul>
|
||||
<li><a href="#streamsapi">2.3 Streams API</a>
|
||||
</ul>
|
||||
</li>
|
||||
<li><a href="#configuration">3. Configuration</a>
|
||||
|
|
|
@ -258,15 +258,15 @@ This quickstart example will demonstrate how to run a streaming application code
|
|||
of the <code>WordCountDemo</code> example code (converted to use Java 8 lambda expressions for easy reading).
|
||||
</p>
|
||||
<pre>
|
||||
KStream<String, Long> wordCounts = textLines
|
||||
// Split each text line, by whitespace, into words.
|
||||
.flatMapValues(value -> Arrays.asList(value.toLowerCase().split("\\W+")))
|
||||
// Ensure the words are available as message keys for the next aggregate operation.
|
||||
.map((key, value) -> new KeyValue<>(value, value))
|
||||
// Count the occurrences of each word (message key).
|
||||
.countByKey(stringSerializer, longSerializer, stringDeserializer, longDeserializer, "Counts")
|
||||
// Convert the resulted aggregate table into another stream.
|
||||
.toStream();
|
||||
KTable<String, Long> wordCounts = textLines
|
||||
// Split each text line, by whitespace, into words.
|
||||
.flatMapValues(value -> Arrays.asList(value.toLowerCase().split("\\W+")))
|
||||
|
||||
// Ensure the words are available as record keys for the next aggregate operation.
|
||||
.map((key, value) -> new KeyValue<>(value, value))
|
||||
|
||||
// Count the occurrences of each word (record key) and store the results into a table named "Counts".
|
||||
.countByKey("Counts")
|
||||
</pre>
|
||||
|
||||
<p>
|
||||
|
|
|
@ -90,6 +90,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> 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>
|
||||
|
|
|
@ -45,7 +45,7 @@ In comparison to log-centric systems like Scribe or Flume, Kafka offers equally
|
|||
|
||||
<h4><a id="uses_streamprocessing" href="#uses_streamprocessing">Stream Processing</a></h4>
|
||||
|
||||
Many users end up doing stage-wise processing of data where data is consumed from topics of raw data and then aggregated, enriched, or otherwise transformed into new Kafka topics for further consumption. For example a processing flow for article recommendation might crawl article content from RSS feeds and publish it to an "articles" topic; further processing might help normalize or deduplicate this content to a topic of cleaned article content; a final stage might attempt to match this content to users. This creates a graph of real-time data flow out of the individual topics. <a href="https://storm.apache.org/">Storm</a> and <a href="http://samza.apache.org/">Samza</a> are popular frameworks for implementing these kinds of transformations.
|
||||
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 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>
|
||||
|
||||
|
|
Loading…
Reference in New Issue