mirror of https://github.com/apache/kafka.git
KAFKA-4112: Remove alpha quality label from Kafka Streams in docs
Rephrase 'alpha quality' wording in Streams section of api.html. Couple of other minor fixes in streams.html Author: Damian Guy <damian.guy@gmail.com> Reviewers: Guozhang Wang, Ismael Juma, Michael G. Noll Closes #1811 from dguy/kstreams-312
This commit is contained in:
parent
3a161db571
commit
268cff7049
|
@ -168,19 +168,20 @@ Examples showing how to use the consumer are given in the
|
||||||
|
|
||||||
<h3><a id="streamsapi" href="#streamsapi">2.3 Streams API</a></h3>
|
<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
|
As of the 0.10.0 release we have added a stream processing engine to Apache Kafka called Kafka Streams, which is a client library that lets users implement their own stream processing applications for data stored in Kafka topics.
|
||||||
applications with data stored in Kafka topics. Kafka Streams is considered alpha quality and its public APIs are likely to change in
|
You can use Kafka Streams from within your Java applications by adding a dependency on the kafka-streams jar using the following maven co-ordinates:
|
||||||
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>
|
<pre>
|
||||||
<dependency>
|
<dependency>
|
||||||
<groupId>org.apache.kafka</groupId>
|
<groupId>org.apache.kafka</groupId>
|
||||||
<artifactId>kafka-streams</artifactId>
|
<artifactId>kafka-streams</artifactId>
|
||||||
<version>0.10.0.0</version>
|
<version>0.10.0.1</version>
|
||||||
</dependency>
|
</dependency>
|
||||||
</pre>
|
</pre>
|
||||||
|
|
||||||
Examples showing how to use this library are given in the
|
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).
|
<a href="http://kafka.apache.org/0100/javadoc/index.html?org/apache/kafka/streams/KafkaStreams.html" title="Kafka 0.10.0 Javadoc">javadocs</a> and <a href="streams.html" title="Kafka Streams Overview">kafka streams overview</a>.
|
||||||
|
<p>
|
||||||
|
Please note that Kafka Streams is a new component of Kafka, and its public APIs may change in future releases.
|
||||||
|
We use the <b>@InterfaceStability.Unstable</b> annotation to denote classes whose APIs may change without backward-compatibility in future releases.
|
||||||
|
</p>
|
|
@ -260,7 +260,7 @@ from a single topic).
|
||||||
KStreamBuilder builder = new KStreamBuilder();
|
KStreamBuilder builder = new KStreamBuilder();
|
||||||
|
|
||||||
KStream<String, GenericRecord> source1 = builder.stream("topic1", "topic2");
|
KStream<String, GenericRecord> source1 = builder.stream("topic1", "topic2");
|
||||||
KTable<String, GenericRecord> source2 = builder.table("topic3");
|
KTable<String, GenericRecord> source2 = builder.table("topic3", "stateStoreName");
|
||||||
</pre>
|
</pre>
|
||||||
|
|
||||||
<h5><a id="streams_dsl_transform" href="#streams_dsl_transform">Transform a stream</a></h5>
|
<h5><a id="streams_dsl_transform" href="#streams_dsl_transform">Transform a stream</a></h5>
|
||||||
|
@ -298,7 +298,7 @@ based on them.
|
||||||
|
|
||||||
<pre>
|
<pre>
|
||||||
// written in Java 8+, using lambda expressions
|
// written in Java 8+, using lambda expressions
|
||||||
KTable<Windowed<String>, Long> counts = source1.groupBykey().aggregate(
|
KTable<Windowed<String>, Long> counts = source1.groupByKey().aggregate(
|
||||||
() -> 0L, // initial value
|
() -> 0L, // initial value
|
||||||
(aggKey, value, aggregate) -> aggregate + 1L, // aggregating value
|
(aggKey, value, aggregate) -> aggregate + 1L, // aggregating value
|
||||||
TimeWindows.of("counts", 5000L).advanceBy(1000L), // intervals in milliseconds
|
TimeWindows.of("counts", 5000L).advanceBy(1000L), // intervals in milliseconds
|
||||||
|
|
Loading…
Reference in New Issue