Skip to content

Commit

Permalink
MINOR: Remove deprecated KafkaStreams constructors in docs (apache#5118)
Browse files Browse the repository at this point in the history
Reviewers: Bill Bejeck <[email protected]>, Matthias J. Sax <[email protected]>
  • Loading branch information
guozhangwang authored Jun 4, 2018
1 parent d02f021 commit 718d6f2
Show file tree
Hide file tree
Showing 11 changed files with 61 additions and 83 deletions.
20 changes: 3 additions & 17 deletions docs/streams/developer-guide/config-streams.html
Original file line number Diff line number Diff line change
Expand Up @@ -34,13 +34,11 @@

<div class="section" id="configuring-a-streams-application">
<span id="streams-developer-guide-configuration"></span><h1>Configuring a Streams Application<a class="headerlink" href="#configuring-a-streams-application" title="Permalink to this headline"></a></h1>
<p>Kafka and Kafka Streams configuration options must be configured before using Streams. You can configure Kafka Streams by specifying parameters in a <code class="docutils literal"><span class="pre">StreamsConfig</span></code> instance.</p>
<p>Kafka and Kafka Streams configuration options must be configured before using Streams. You can configure Kafka Streams by specifying parameters in a <code class="docutils literal"><span class="pre">java.util.Properties</span></code> instance.</p>
<ol class="arabic">
<li><p class="first">Create a <code class="docutils literal"><span class="pre">java.util.Properties</span></code> instance.</p>
</li>
<li><p class="first">Set the <a class="reference internal" href="#streams-developer-guide-required-configs"><span class="std std-ref">parameters</span></a>.</p>
</li>
<li><p class="first">Construct a <code class="docutils literal"><span class="pre">StreamsConfig</span></code> instance from the <code class="docutils literal"><span class="pre">Properties</span></code> instance. For example:</p>
<li><p class="first">Set the <a class="reference internal" href="#streams-developer-guide-required-configs"><span class="std std-ref">parameters</span></a>. For example:</p>
<div class="highlight-java"><div class="highlight"><pre><span></span><span class="kn">import</span> <span class="nn">java.util.Properties</span><span class="o">;</span>
<span class="kn">import</span> <span class="nn">org.apache.kafka.streams.StreamsConfig</span><span class="o">;</span>

Expand All @@ -50,9 +48,6 @@
<span class="n">settings</span><span class="o">.</span><span class="na">put</span><span class="o">(</span><span class="n">StreamsConfig</span><span class="o">.</span><span class="na">BOOTSTRAP_SERVERS_CONFIG</span><span class="o">,</span> <span class="s">&quot;kafka-broker1:9092&quot;</span><span class="o">);</span>
<span class="c1">// Any further settings</span>
<span class="n">settings</span><span class="o">.</span><span class="na">put</span><span class="o">(...</span> <span class="o">,</span> <span class="o">...);</span>

<span class="c1">// Create an instance of StreamsConfig from the Properties instance</span>
<span class="n">StreamsConfig</span> <span class="n">config</span> <span class="o">=</span> <span class="k">new</span> <span class="n">StreamsConfig</span><span class="o">(</span><span class="n">settings</span><span class="o">);</span>
</pre></div>
</div>
</li>
Expand Down Expand Up @@ -520,15 +515,13 @@ <h4><a class="toc-backref" href="#id14">state.dir</a><a class="headerlink" href=
</div>
<div class="section" id="kafka-consumers-and-producer-configuration-parameters">
<h3><a class="toc-backref" href="#id16">Kafka consumers and producer configuration parameters</a><a class="headerlink" href="#kafka-consumers-and-producer-configuration-parameters" title="Permalink to this headline"></a></h3>
<p>You can specify parameters for the Kafka <a class="reference external" href="../../../javadoc/org/apache/kafka/clients/consumer/package-summary.html">consumers</a> and <a class="reference external" href="../../../javadoc/org/apache/kafka/clients/producer/package-summary.html">producers</a> that are used internally. The consumer and producer settings
are defined by specifying parameters in a <code class="docutils literal"><span class="pre">StreamsConfig</span></code> instance.</p>
<p>You can specify parameters for the Kafka <a class="reference external" href="../../../javadoc/org/apache/kafka/clients/consumer/package-summary.html">consumers</a> and <a class="reference external" href="../../../javadoc/org/apache/kafka/clients/producer/package-summary.html">producers</a> that are used internally.
<p>In this example, the Kafka <a class="reference external" href="../../../javadoc/org/apache/kafka/clients/consumer/ConsumerConfig.html#SESSION_TIMEOUT_MS_CONFIG">consumer session timeout</a> is configured to be 60000 milliseconds in the Streams settings:</p>
<div class="highlight-java"><div class="highlight"><pre><span></span><span class="n">Properties</span> <span class="n">streamsSettings</span> <span class="o">=</span> <span class="k">new</span> <span class="n">Properties</span><span class="o">();</span>
<span class="c1">// Example of a &quot;normal&quot; setting for Kafka Streams</span>
<span class="n">streamsSettings</span><span class="o">.</span><span class="na">put</span><span class="o">(</span><span class="n">StreamsConfig</span><span class="o">.</span><span class="na">BOOTSTRAP_SERVERS_CONFIG</span><span class="o">,</span> <span class="s">&quot;kafka-broker-01:9092&quot;</span><span class="o">);</span>
<span class="c1">// Customize the Kafka consumer settings of your Streams application</span>
<span class="n">streamsSettings</span><span class="o">.</span><span class="na">put</span><span class="o">(</span><span class="n">ConsumerConfig</span><span class="o">.</span><span class="na">SESSION_TIMEOUT_MS_CONFIG</span><span class="o">,</span> <span class="mi">60000</span><span class="o">);</span>
<span class="n">StreamsConfig</span> <span class="n">config</span> <span class="o">=</span> <span class="k">new</span> <span class="n">StreamsConfig</span><span class="o">(</span><span class="n">streamsSettings</span><span class="o">);</span>
</pre></div>
</div>
<div class="section" id="naming">
Expand Down Expand Up @@ -706,18 +699,11 @@ <h4><a class="toc-backref" href="#id22">acks</a><a class="headerlink" href="#ack
<h4><a class="toc-backref" href="#id23">replication.factor</a><a class="headerlink" href="#id2" title="Permalink to this headline"></a></h4>
<blockquote>
<div>See the <a class="reference internal" href="#replication-factor-parm"><span class="std std-ref">description here</span></a>.</div></blockquote>
<p>You define these settings via <code class="docutils literal"><span class="pre">StreamsConfig</span></code>:</p>
<div class="highlight-java"><div class="highlight"><pre><span></span><span class="n">Properties</span> <span class="n">streamsSettings</span> <span class="o">=</span> <span class="k">new</span> <span class="n">Properties</span><span class="o">();</span>
<span class="n">streamsSettings</span><span class="o">.</span><span class="na">put</span><span class="o">(</span><span class="n">StreamsConfig</span><span class="o">.</span><span class="na">REPLICATION_FACTOR_CONFIG</span><span class="o">,</span> <span class="mi">3</span><span class="o">);</span>
<span class="n">streamsSettings</span><span class="o">.</span><span class="na">put</span><span class="o">(</span><span class="n">StreamsConfig</span><span class="o">.</span><span class="na">producerPrefix</span><span class="o">(</span><span class="n">ProducerConfig</span><span class="o">.</span><span class="na">ACKS_CONFIG</span><span class="o">),</span> <span class="s">&quot;all&quot;</span><span class="o">);</span>
</pre></div>
</div>
<div class="admonition note">
<p class="first admonition-title">Note</p>
<p class="last">A future version of Kafka Streams will allow developers to set their own app-specific configuration settings through
<code class="docutils literal"><span class="pre">StreamsConfig</span></code> as well, which can then be accessed through
<a class="reference external" href="../../../javadoc/org/apache/kafka/streams/processor/ProcessorContext.html">ProcessorContext</a>.</p>
</div>
</div>
</div>
</div>
Expand Down
6 changes: 2 additions & 4 deletions docs/streams/developer-guide/datatypes.html
Original file line number Diff line number Diff line change
Expand Up @@ -36,7 +36,7 @@
<p>Every Kafka Streams application must provide SerDes (Serializer/Deserializer) for the data types of record keys and record values (e.g. <code class="docutils literal"><span class="pre">java.lang.String</span></code>) to materialize the data when necessary. Operations that require such SerDes information include: <code class="docutils literal"><span class="pre">stream()</span></code>, <code class="docutils literal"><span class="pre">table()</span></code>, <code class="docutils literal"><span class="pre">to()</span></code>, <code class="docutils literal"><span class="pre">through()</span></code>, <code class="docutils literal"><span class="pre">groupByKey()</span></code>, <code class="docutils literal"><span class="pre">groupBy()</span></code>.</p>
<p>You can provide SerDes by using either of these methods:</p>
<ul class="simple">
<li>By setting default SerDes via a <code class="docutils literal"><span class="pre">StreamsConfig</span></code> instance.</li>
<li>By setting default SerDes in the <code class="docutils literal"><span class="pre">java.util.Properties</span></code> config instance.</li>
<li>By specifying explicit SerDes when calling the appropriate API methods, thus overriding the defaults.</li>
</ul>

Expand All @@ -55,7 +55,7 @@
</ul>
<div class="section" id="configuring-serdes">
<h2>Configuring SerDes<a class="headerlink" href="#configuring-serdes" title="Permalink to this headline"></a></h2>
<p>SerDes specified in the Streams configuration via <code class="docutils literal"><span class="pre">StreamsConfig</span></code> are used as the default in your Kafka Streams application.</p>
<p>SerDes specified in the Streams configuration are used as the default in your Kafka Streams application.</p>
<div class="highlight-java"><div class="highlight"><pre><span></span><span class="kn">import</span> <span class="nn">org.apache.kafka.common.serialization.Serdes</span><span class="o">;</span>
<span class="kn">import</span> <span class="nn">org.apache.kafka.streams.StreamsConfig</span><span class="o">;</span>

Expand All @@ -64,8 +64,6 @@ <h2>Configuring SerDes<a class="headerlink" href="#configuring-serdes" title="Pe
<span class="n">settings</span><span class="o">.</span><span class="na">put</span><span class="o">(</span><span class="n">StreamsConfig</span><span class="o">.</span><span class="na">KEY_SERDE_CLASS_CONFIG</span><span class="o">,</span> <span class="n">Serdes</span><span class="o">.</span><span class="na">String</span><span class="o">().</span><span class="na">getClass</span><span class="o">().</span><span class="na">getName</span><span class="o">());</span>
<span class="c1">// Default serde for values of data records (here: built-in serde for Long type)</span>
<span class="n">settings</span><span class="o">.</span><span class="na">put</span><span class="o">(</span><span class="n">StreamsConfig</span><span class="o">.</span><span class="na">VALUE_SERDE_CLASS_CONFIG</span><span class="o">,</span> <span class="n">Serdes</span><span class="o">.</span><span class="na">Long</span><span class="o">().</span><span class="na">getClass</span><span class="o">().</span><span class="na">getName</span><span class="o">());</span>

<span class="n">StreamsConfig</span> <span class="n">config</span> <span class="o">=</span> <span class="k">new</span> <span class="n">StreamsConfig</span><span class="o">(</span><span class="n">settings</span><span class="o">);</span>
</pre></div>
</div>
</div>
Expand Down
10 changes: 5 additions & 5 deletions docs/streams/developer-guide/dsl-api.html
Original file line number Diff line number Diff line change
Expand Up @@ -3044,14 +3044,14 @@ <h2><a class="toc-backref" href="#id7">Overview</a><a class="headerlink" href="#


<span class="c1">// Write the stream to the output topic, using the configured default key</span>
<span class="c1">// and value serdes of your `StreamsConfig`.</span>
<span class="c1">// and value serdes.</span>
<span class="n">stream</span><span class="o">.</span><span class="na">to</span><span class="o">(</span><span class="s">&quot;my-stream-output-topic&quot;</span><span class="o">);</span>

<span class="c1">// Same for table</span>
<span class="n">table</span><span class="o">.</span><span class="na">to</span><span class="o">(</span><span class="s">&quot;my-table-output-topic&quot;</span><span class="o">);</span>

<span class="c1">// Write the stream to the output topic, using explicit key and value serdes,</span>
<span class="c1">// (thus overriding the defaults of your `StreamsConfig`).</span>
<span class="c1">// (thus overriding the defaults in the config properties).</span>
<span class="n">stream</span><span class="o">.</span><span class="na">to</span><span class="o">(</span><span class="s">&quot;my-stream-output-topic&quot;</span><span class="o">,</span> <span class="n">Produced</span><span class="o">.</span><span class="na">with</span><span class="o">(</span><span class="n">Serdes</span><span class="o">.</span><span class="na">String</span><span class="o">(),</span> <span class="n">Serdes</span><span class="o">.</span><span class="na">Long</span><span class="o">());</span>
</pre></div>
</div>
Expand Down Expand Up @@ -3131,7 +3131,7 @@ <h2><a class="toc-backref" href="#id7">Overview</a><a class="headerlink" href="#
</div>
</div>
<div class="section" id="testing-a-streams-app">
<a class="headerlink" href="#testing-a-streams-app" title="Permalink to this headline"><h2>Testing a Streams application</a></h2>
<a class="headerlink" href="#testing-a-streams-app" title="Permalink to this headline"><h2>Testing a Streams application</h2></a>
Kafka Streams comes with a <code>test-utils</code> module to help you test your application <a href="testing.html">here</a>.
</div>
</div>
Expand Down Expand Up @@ -3201,7 +3201,7 @@ <h2><a class="toc-backref" href="#id7">Overview</a><a class="headerlink" href="#
object WordCountApplication extends App {
import Serdes._

val config: Properties = {
val props: Properties = {
val p = new Properties()
p.put(StreamsConfig.APPLICATION_ID_CONFIG, "wordcount-application")
p.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "kafka-broker1:9092")
Expand All @@ -3216,7 +3216,7 @@ <h2><a class="toc-backref" href="#id7">Overview</a><a class="headerlink" href="#
.count(Materialized.as("counts-store"))
wordCounts.toStream.to("WordsWithCountsTopic")

val streams: KafkaStreams = new KafkaStreams(builder.build(), config)
val streams: KafkaStreams = new KafkaStreams(builder.build(), props)
streams.start()

sys.ShutdownHookThread {
Expand Down
Loading

0 comments on commit 718d6f2

Please sign in to comment.