mirror of https://github.com/apache/kafka.git
MINOR: Kafka Streams doc updates for 4.0 release (#18488)
Reviewers: Bill Bejeck <bill@confluent.io>
This commit is contained in:
parent
47f22faac3
commit
3f3bc90509
|
@ -67,7 +67,7 @@
|
|||
<p><b>Tip</b></p>
|
||||
<p class="last"><strong>Combining the DSL and the Processor API:</strong>
|
||||
You can combine the convenience of the DSL with the power and flexibility of the Processor API as described in the
|
||||
section <a class="reference internal" href="dsl-api.html#streams-developer-guide-dsl-process"><span class="std std-ref">Applying processors and transformers (Processor API integration)</span></a>.</p>
|
||||
section <a class="reference internal" href="dsl-api.html#streams-developer-guide-dsl-process"><span class="std std-ref">Applying processors (Processor API integration)</span></a>.</p>
|
||||
</div>
|
||||
<p>For a complete list of available API functionality, see the <a href="/{{version}}/javadoc/org/apache/kafka/streams/package-summary.html">Streams</a> API docs.</p>
|
||||
</div>
|
||||
|
@ -82,27 +82,25 @@
|
|||
phase. Processor instances should perform any required initialization in this method. The <code class="docutils literal"><span class="pre">init()</span></code> method passes in a <code class="docutils literal"><span class="pre">ProcessorContext</span></code>
|
||||
instance, which provides access to the metadata of the currently processed record, including its source Kafka topic and partition,
|
||||
its corresponding message offset, and further such information. You can also use this context instance to schedule a punctuation
|
||||
function (via <code class="docutils literal"><span class="pre">ProcessorContext#schedule()</span></code>), to forward a new record as a key-value pair to the downstream processors (via <code class="docutils literal"><span class="pre">ProcessorContext#forward()</span></code>),
|
||||
and to commit the current processing progress (via <code class="docutils literal"><span class="pre">ProcessorContext#commit()</span></code>).
|
||||
function (via <code class="docutils literal"><span class="pre">ProcessorContext#schedule()</span></code>), to forward a new record to the downstream processors (via <code class="docutils literal"><span class="pre">ProcessorContext#forward()</span></code>),
|
||||
and to request a commit of the current processing progress (via <code class="docutils literal"><span class="pre">ProcessorContext#commit()</span></code>).
|
||||
Any resources you set up in <code class="docutils literal"><span class="pre">init()</span></code> can be cleaned up in the
|
||||
<code class="docutils literal"><span class="pre">close()</span></code> method. Note that Kafka Streams may re-use a single
|
||||
<code class="docutils literal"><span class="pre">Processor</span></code> object by calling
|
||||
<code class="docutils literal"><span class="pre">init()</span></code> on it again after <code class="docutils literal"><span class="pre">close()</span></code>.</p>
|
||||
<p>
|
||||
The <code class="docutils literal"><span class="pre">Processor</span></code> interface takes two sets of generic parameters:
|
||||
<p>The <code class="docutils literal"><span class="pre">Processor</span></code> interface takes four generic parameters:
|
||||
<code class="docutils literal"><span class="pre">KIn, VIn, KOut, VOut</span></code>. These define the input and output types
|
||||
that the processor implementation can handle. <code class="docutils literal"><span class="pre">KIn</span></code> and
|
||||
<code class="docutils literal"><span class="pre">VIn</span></code> define the key and value types that will be passed
|
||||
<code class="docutils literal"><span class="pre">VIn</span></code> define the key and value types of the <code class="docutils literal"><span class="pre">Record</span></code> that will be passed
|
||||
to <code class="docutils literal"><span class="pre">process()</span></code>.
|
||||
Likewise, <code class="docutils literal"><span class="pre">KOut</span></code> and <code class="docutils literal"><span class="pre">VOut</span></code>
|
||||
define the forwarded key and value types that <code class="docutils literal"><span class="pre">ProcessorContext#forward()</span></code>
|
||||
define the forwarded key and value types for the result <code class="docutils literal"><span class="pre">Record</span></code> that <code class="docutils literal"><span class="pre">ProcessorContext#forward()</span></code>
|
||||
will accept. If your processor does not forward any records at all (or if it only forwards
|
||||
<code class="docutils literal"><span class="pre">null</span></code> keys or values),
|
||||
a best practice is to set the output generic type argument to
|
||||
<code class="docutils literal"><span class="pre">Void</span></code>.
|
||||
If it needs to forward multiple types that don't share a common superclass, you will
|
||||
have to set the output generic type argument to <code class="docutils literal"><span class="pre">Object</span></code>.
|
||||
</p>
|
||||
have to set the output generic type argument to <code class="docutils literal"><span class="pre">Object</span></code>.</p>
|
||||
<p>
|
||||
Both the <code class="docutils literal"><span class="pre">Processor#process()</span></code>
|
||||
and the <code class="docutils literal"><span class="pre">ProcessorContext#forward()</span></code>
|
||||
|
@ -120,10 +118,8 @@
|
|||
Note that this does not mutate <code class="docutils literal"><span class="pre">inputRecord</span></code>,
|
||||
but instead creates a shallow copy. Beware that this is only a shallow copy, so if you
|
||||
plan to mutate the key, value, or headers elsewhere in the program, you will want to
|
||||
create a deep copy of those fields yourself.
|
||||
</p>
|
||||
<p>
|
||||
In addition to handling incoming records via
|
||||
create a deep copy of those fields yourself.</p>
|
||||
<p>In addition to handling incoming records via
|
||||
<code class="docutils literal"><span class="pre">Processor#process()</span></code>,
|
||||
you have the option to schedule periodic invocation (called "punctuation")
|
||||
in your processor's <code class="docutils literal"><span class="pre">init()</span></code>
|
||||
|
@ -216,8 +212,8 @@
|
|||
</div>
|
||||
<div class="section" id="state-stores">
|
||||
<span id="streams-developer-guide-state-store"></span><h2><a class="toc-backref" href="#id3">State Stores</a><a class="headerlink" href="#state-stores" title="Permalink to this headline"></a></h2>
|
||||
<p>To implement a <strong>stateful</strong> <code class="docutils literal"><span class="pre">Processor</span></code> or <code class="docutils literal"><span class="pre">Transformer</span></code>, you must provide one or more state stores to the processor
|
||||
or transformer (<em>stateless</em> processors or transformers do not need state stores). State stores can be used to remember
|
||||
<p>To implement a <strong>stateful</strong> <code class="docutils literal"><span class="pre">Processor</span></code>, you must provide one or more state stores to the processor
|
||||
(<em>stateless</em> processors do not need state stores). State stores can be used to remember
|
||||
recently received input records, to track rolling aggregates, to de-duplicate input records, and more.
|
||||
Another feature of state stores is that they can be
|
||||
<a class="reference internal" href="interactive-queries.html#streams-developer-guide-interactive-queries"><span class="std std-ref">interactively queried</span></a> from other applications, such as a
|
||||
|
@ -499,15 +495,9 @@ StoreBuilder<KeyValueStore<String, Long>> countStoreSupplier = Store
|
|||
<p>As we have mentioned in the <a href=#defining-a-stream-processor>Defining a Stream Processor</a> section, a <code>ProcessorContext</code> control the processing workflow, such as scheduling a punctuation function, and committing the current processed state.</p>
|
||||
<p>This object can also be used to access the metadata related with the application like
|
||||
<code class="docutils literal"><span class="pre">applicationId</span></code>, <code class="docutils literal"><span class="pre">taskId</span></code>,
|
||||
and <code class="docutils literal"><span class="pre">stateDir</span></code>, and also record related metadata as <code class="docutils literal"><span class="pre">topic</span></code>,
|
||||
<code class="docutils literal"><span class="pre">partition</span></code>, <code class="docutils literal"><span class="pre">offset</span></code>, <code class="docutils literal"><span class="pre">timestamp</span></code> and
|
||||
<code class="docutils literal"><span class="pre">headers</span></code>.</p>
|
||||
<p>Here is an example implementation of how to add a new header to the record:</p>
|
||||
<pre class="line-numbers"><code class="language-java">public void process(String key, String value) {
|
||||
|
||||
// add a header to the elements
|
||||
context().headers().add.("key", "value");
|
||||
}</code></pre>
|
||||
and <code class="docutils literal"><span class="pre">stateDir</span></code>, and also <code class="docutils literal"><span class="pre">RecordMetadata</span></code> such as
|
||||
<code class="docutils literal"><span class="pre">topic</span></code>,
|
||||
<code class="docutils literal"><span class="pre">partition</span></code>, and <code class="docutils literal"><span class="pre">offset</span></code>.</p>
|
||||
<div class="section" id="connecting-processors-and-state-stores">
|
||||
<h2><a class="toc-backref" href="#id8">Connecting Processors and State Stores</a><a class="headerlink" href="#connecting-processors-and-state-stores" title="Permalink to this headline"></a></h2>
|
||||
<p>Now that a <a class="reference internal" href="#streams-developer-guide-stream-processor"><span class="std std-ref">processor</span></a> (WordCountProcessor) and the
|
||||
|
@ -572,7 +562,7 @@ builder.addSource("Source", "source-topic")
|
|||
upstream processor of the <code class="docutils literal"><span class="pre">"Sink"</span></code> node. As a result, whenever the <code class="docutils literal"><span class="pre">"Source"</span></code> node forwards a newly fetched record from
|
||||
Kafka to its downstream <code class="docutils literal"><span class="pre">"Process"</span></code> node, the <code class="docutils literal"><span class="pre">WordCountProcessor#process()</span></code> method is triggered to process the record and
|
||||
update the associated state store. Whenever <code class="docutils literal"><span class="pre">context#forward()</span></code> is called in the
|
||||
<code class="docutils literal"><span class="pre">WordCountProcessor#punctuate()</span></code> method, the aggregate key-value pair will be sent via the <code class="docutils literal"><span class="pre">"Sink"</span></code> processor node to
|
||||
<code class="docutils literal"><span class="pre">WordCountProcessor#punctuate()</span></code> method, the aggregate records will be sent via the <code class="docutils literal"><span class="pre">"Sink"</span></code> processor node to
|
||||
the Kafka topic <code class="docutils literal"><span class="pre">"sink-topic"</span></code>. Note that in the <code class="docutils literal"><span class="pre">WordCountProcessor</span></code> implementation, you must refer to the
|
||||
same store name <code class="docutils literal"><span class="pre">"Counts"</span></code> when accessing the key-value store, otherwise an exception will be thrown at runtime,
|
||||
indicating that the state store cannot be found. If the state store is not associated with the processor
|
||||
|
|
|
@ -284,7 +284,7 @@ processorUnderTest.init(context);</code></pre>
|
|||
props.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.String().getClass());
|
||||
props.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.Long().getClass());
|
||||
props.put("some.other.config", "some config value");
|
||||
final MockProcessorContext<String, Long> context = new MockProcessorContext<>(props);
|
||||
final MockProcessorContext<String, Long> context = new MockProcessorContext<>(props);</code></pre>
|
||||
</p>
|
||||
<b>Captured data</b>
|
||||
<p>
|
||||
|
|
|
@ -25,6 +25,7 @@ import org.apache.kafka.streams.kstream.KGroupedTable;
|
|||
import org.apache.kafka.streams.kstream.KStream;
|
||||
import org.apache.kafka.streams.kstream.KTable;
|
||||
import org.apache.kafka.streams.kstream.Materialized;
|
||||
import org.apache.kafka.streams.kstream.ValueTransformerWithKeySupplier;
|
||||
import org.apache.kafka.streams.kstream.internals.ConsumedInternal;
|
||||
import org.apache.kafka.streams.kstream.internals.InternalStreamsBuilder;
|
||||
import org.apache.kafka.streams.kstream.internals.MaterializedInternal;
|
||||
|
@ -510,8 +511,8 @@ public class StreamsBuilder {
|
|||
* Adds a state store to the underlying {@link Topology}.
|
||||
* <p>
|
||||
* It is required to connect state stores to {@link org.apache.kafka.streams.processor.api.Processor Processors},
|
||||
* {@link org.apache.kafka.streams.kstream.Transformer Transformers},
|
||||
* or {@link org.apache.kafka.streams.kstream.ValueTransformer ValueTransformers} before they can be used.
|
||||
* or {@link org.apache.kafka.streams.kstream.KTable#transformValues(ValueTransformerWithKeySupplier, String...) ValueTransformers}
|
||||
* before they can be used.
|
||||
*
|
||||
* @param builder the builder used to obtain this state store {@link StateStore} instance
|
||||
* @return itself
|
||||
|
@ -540,8 +541,7 @@ public class StreamsBuilder {
|
|||
* The default {@link TimestampExtractor} as specified in the {@link StreamsConfig config} is used.
|
||||
* <p>
|
||||
* It is not required to connect a global store to the {@link Processor Processors},
|
||||
* {@link org.apache.kafka.streams.kstream.Transformer Transformers},
|
||||
* or {@link org.apache.kafka.streams.kstream.ValueTransformer ValueTransformer};
|
||||
* or {@link org.apache.kafka.streams.kstream.KTable#transformValues(ValueTransformerWithKeySupplier, String...) ValueTransformer};
|
||||
* those have read-only access to all global stores by default.
|
||||
*
|
||||
* @param storeBuilder user defined {@link StoreBuilder}; can't be {@code null}
|
||||
|
|
|
@ -32,7 +32,7 @@ import java.util.regex.Pattern;
|
|||
* In contrast, two sub-topologies are not connected but can be linked to each other via topics, i.e., if one
|
||||
* sub-topology {@link Topology#addSink(String, String, String...) writes} into a topic and another sub-topology
|
||||
* {@link Topology#addSource(String, String...) reads} from the same topic.
|
||||
* Message {@link ProcessorContext#forward(Record, String) forwards} using custom Processors and Transformers are not considered in the topology graph.
|
||||
* Message {@link ProcessorContext#forward(Record, String) forwards} using custom Processors are not considered in the topology graph.
|
||||
* <p>
|
||||
* When {@link KafkaStreams#start()} is called, different sub-topologies will be constructed and executed as independent
|
||||
* {@link StreamTask tasks}.
|
||||
|
|
|
@ -16,20 +16,20 @@
|
|||
*/
|
||||
package org.apache.kafka.streams.kstream;
|
||||
|
||||
import org.apache.kafka.streams.processor.api.FixedKeyProcessor;
|
||||
|
||||
/**
|
||||
* The {@code ValueMapper} interface for mapping a value to a new value of arbitrary type.
|
||||
* This is a stateless record-by-record operation, i.e, {@link #apply(Object)} is invoked individually for each record
|
||||
* of a stream (cf. {@link ValueTransformer} for stateful value transformation).
|
||||
* If {@code ValueMapper} is applied to a {@link org.apache.kafka.streams.KeyValue key-value pair} record the record's
|
||||
* of a stream (cf. {@link org.apache.kafka.streams.processor.api.FixedKeyProcessor} for stateful value transformation).
|
||||
* If {@code ValueMapper} is applied to a {@link org.apache.kafka.streams.processor.api.Record} the record's
|
||||
* key is preserved.
|
||||
* If a record's key and value should be modified {@link KeyValueMapper} can be used.
|
||||
*
|
||||
* @param <V> value type
|
||||
* @param <VR> mapped value type
|
||||
* @see KeyValueMapper
|
||||
* @see ValueTransformer
|
||||
* @see ValueTransformerWithKey
|
||||
* @see FixedKeyProcessor
|
||||
* @see KStream#mapValues(ValueMapper)
|
||||
* @see KStream#mapValues(ValueMapperWithKey)
|
||||
* @see KStream#flatMapValues(ValueMapper)
|
||||
|
|
|
@ -19,8 +19,8 @@ package org.apache.kafka.streams.kstream;
|
|||
/**
|
||||
* The {@code ValueMapperWithKey} interface for mapping a value to a new value of arbitrary type.
|
||||
* This is a stateless record-by-record operation, i.e, {@link #apply(Object, Object)} is invoked individually for each
|
||||
* record of a stream (cf. {@link ValueTransformer} for stateful value transformation).
|
||||
* If {@code ValueMapperWithKey} is applied to a {@link org.apache.kafka.streams.KeyValue key-value pair} record the
|
||||
* record of a stream (cf. {@link org.apache.kafka.streams.processor.api.FixedKeyProcessor} for stateful value transformation).
|
||||
* If {@code ValueMapperWithKey} is applied to a {@link org.apache.kafka.streams.processor.api.Record} the
|
||||
* record's key is preserved.
|
||||
* Note that the key is read-only and should not be modified, as this can lead to corrupt partitioning.
|
||||
* If a record's key and value should be modified {@link KeyValueMapper} can be used.
|
||||
|
@ -29,8 +29,7 @@ package org.apache.kafka.streams.kstream;
|
|||
* @param <V> value type
|
||||
* @param <VR> mapped value type
|
||||
* @see KeyValueMapper
|
||||
* @see ValueTransformer
|
||||
* @see ValueTransformerWithKey
|
||||
* @see org.apache.kafka.streams.processor.api.FixedKeyProcessor
|
||||
* @see KStream#mapValues(ValueMapper)
|
||||
* @see KStream#mapValues(ValueMapperWithKey)
|
||||
* @see KStream#flatMapValues(ValueMapper)
|
||||
|
|
|
@ -110,9 +110,10 @@ public interface ProcessorContext {
|
|||
<S extends StateStore> S getStateStore(final String name);
|
||||
|
||||
/**
|
||||
* Schedule a periodic operation for processors. A processor may call this method during
|
||||
* {@link org.apache.kafka.streams.kstream.ValueTransformer#init(ProcessorContext) initialization} or
|
||||
* {@link org.apache.kafka.streams.kstream.ValueTransformer#transform(Object) processing} to
|
||||
* Schedule a periodic operation for processors. A processor may call this method during a
|
||||
* {@link org.apache.kafka.streams.kstream.KTable#transformValues(ValueTransformerWithKeySupplier, String...)}'s
|
||||
* {@link org.apache.kafka.streams.kstream.ValueTransformerWithKey#init(ProcessorContext) initialization} or
|
||||
* {@link org.apache.kafka.streams.kstream.ValueTransformerWithKey#transform(Object, Object) processing} to
|
||||
* schedule a periodic callback — called a punctuation — to {@link Punctuator#punctuate(long)}.
|
||||
* The type parameter controls what notion of time is used for punctuation:
|
||||
* <ul>
|
||||
|
|
|
@ -42,8 +42,7 @@ public final class ForwardingDisabledProcessorContext implements ProcessorContex
|
|||
|
||||
private static final String EXPLANATION = "ProcessorContext#forward() is not supported from this context, "
|
||||
+ "as the framework must ensure the key is not changed (#forward allows changing the key on "
|
||||
+ "messages which are sent). Try another function, which doesn't allow the key to be changed "
|
||||
+ "(for example - #transformValues).";
|
||||
+ "messages which are sent). Use KStream.process() if you need to change the key.";
|
||||
|
||||
public ForwardingDisabledProcessorContext(final ProcessorContext delegate) {
|
||||
this.delegate = Objects.requireNonNull(delegate, "delegate");
|
||||
|
|
|
@ -174,7 +174,7 @@ public final class ProcessorContextImpl extends AbstractProcessorContext<Object,
|
|||
" as the store is not connected to the processor. If you add stores manually via '.addStateStore()' " +
|
||||
"make sure to connect the added store to the processor by providing the processor name to " +
|
||||
"'.addStateStore()' or connect them via '.connectProcessorAndStateStores()'. " +
|
||||
"DSL users need to provide the store name to '.process()', '.transform()', or '.transformValues()' " +
|
||||
"DSL users need to provide the store name to '.process()', '.processValues()', or '.transformValues()' " +
|
||||
"to connect the store to the corresponding operator, or they can provide a StoreBuilder by implementing " +
|
||||
"the stores() method on the Supplier itself. If you do not add stores manually, " +
|
||||
"please file a bug report at https://issues.apache.org/jira/projects/KAFKA.");
|
||||
|
@ -236,7 +236,7 @@ public final class ProcessorContextImpl extends AbstractProcessorContext<Object,
|
|||
final ProcessorNode<?, ?, ?, ?> previousNode = currentNode();
|
||||
if (previousNode == null) {
|
||||
throw new StreamsException("Current node is unknown. This can happen if 'forward()' is called " +
|
||||
"in an illegal scope. The root cause could be that a 'Processor' or 'Transformer' instance" +
|
||||
"in an illegal scope. The root cause could be that a 'Processor' instance " +
|
||||
"is shared. To avoid this error, make sure that your suppliers return new instances " +
|
||||
"each time 'get()' of Supplier is called and do not return the same object reference " +
|
||||
"multiple times.");
|
||||
|
|
Loading…
Reference in New Issue