Author: Damian Guy <damian.guy@gmail.com>
Reviewers: Michael G. Noll <michael@confluent.io>, Bill Bejeck <bill@confluent.io>, Matthias J. Sax <matthias@confluent.io>, Ismael Juma <ismael@juma.me.uk>
Closes#3971 from dguy/kip-182-docs
@ -1383,65 +1383,72 @@ Note that in the <code>WordCountProcessor</code> implementation, users need to r
@@ -1383,65 +1383,72 @@ Note that in the <code>WordCountProcessor</code> implementation, users need to r
// Java 8+ examples, using lambda expressions
// Aggregating with time-based windowing (here: with 5-minute tumbling windows)
new Aggregator<String, Long, Long>() { /* adder */
@Override
public Long apply(String aggKey, Long newValue, Long aggValue) {
return aggValue + newValue;
}
},
new Merger<String, Long>() { /* session merger */
@Override
public Long apply(String aggKey, Long leftAggValue, Long rightAggValue) {
return rightAggValue + leftAggValue;
}
},
Materialized.<String, Long, SessionStore<Bytes, byte[]>>as("sessionized-aggregated-stream-store") /* state store name */
.withValueSerde(Serdes.Long()) /* serde for aggregate value */
);
</pre>
<p>
@ -1478,12 +1485,10 @@ Note that in the <code>WordCountProcessor</code> implementation, users need to r
@@ -1478,12 +1485,10 @@ Note that in the <code>WordCountProcessor</code> implementation, users need to r
Detailed behavior for <code>KGroupedStream</code>:
@ -1518,14 +1523,14 @@ Note that in the <code>WordCountProcessor</code> implementation, users need to r
@@ -1518,14 +1523,14 @@ Note that in the <code>WordCountProcessor</code> implementation, users need to r
@ -1561,14 +1566,14 @@ Note that in the <code>WordCountProcessor</code> implementation, users need to r
@@ -1561,14 +1566,14 @@ Note that in the <code>WordCountProcessor</code> implementation, users need to r
@ -1580,8 +1585,8 @@ Note that in the <code>WordCountProcessor</code> implementation, users need to r
@@ -1580,8 +1585,8 @@ Note that in the <code>WordCountProcessor</code> implementation, users need to r
@ -1596,8 +1601,8 @@ Note that in the <code>WordCountProcessor</code> implementation, users need to r
@@ -1596,8 +1601,8 @@ Note that in the <code>WordCountProcessor</code> implementation, users need to r
public Long apply(Long aggValue, Long oldValue) {
return aggValue - oldValue;
}
},
"reduced-table-store" /* state store name */);
}
);
</pre>
<p>
Detailed behavior for <code>KGroupedStream</code>:
@ -1659,41 +1664,39 @@ Note that in the <code>WordCountProcessor</code> implementation, users need to r
@@ -1659,41 +1664,39 @@ Note that in the <code>WordCountProcessor</code> implementation, users need to r
// Java 8+ examples, using lambda expressions
// Aggregating with time-based windowing (here: with 5-minute tumbling windows)
@ -1723,16 +1726,22 @@ Note that in the <code>WordCountProcessor</code> implementation, users need to r
@@ -1723,16 +1726,22 @@ Note that in the <code>WordCountProcessor</code> implementation, users need to r
</p>
<preclass="brush: java;">
// Key: word, value: count
Properties streamsProperties == ...;
// specify the default serdes so we don't need to elsewhere.
@ -1836,8 +1845,10 @@ Note that in the <code>WordCountProcessor</code> implementation, users need to r
@@ -1836,8 +1845,10 @@ Note that in the <code>WordCountProcessor</code> implementation, users need to r
.withKeySerde(Serdes.String() /* serde for aggregate key */)
.withValueSerde(Serdes.Long() /* serde for aggregate value */)
);
</pre>
<p>
<b>Impact of <ahref=#streams_developer-guide_memory-management_record-cache>record caches</a></b>:
@ -2253,7 +2264,10 @@ Note that in the <code>WordCountProcessor</code> implementation, users need to r
@@ -2253,7 +2264,10 @@ Note that in the <code>WordCountProcessor</code> implementation, users need to r
.groupBy((key, word) -> word, Serialized.with(stringSerde, stringSerde));
// Create a window state store named "CountsWindowStore" that contains the word counts for every minute
withKeySerde(Serdes.String()); // count() sets value serde to Serdes.Long() automatically
);
</pre>
<p>
@ -2396,14 +2410,14 @@ Note that in the <code>WordCountProcessor</code> implementation, users need to r
@@ -2396,14 +2410,14 @@ Note that in the <code>WordCountProcessor</code> implementation, users need to r
Topology topology = ...;
ProcessorSupplier processorSuppler = ...;
// Create CustomStoreSupplier for store name the-custom-store
MyCustomStoreSuppler customStoreSupplier = new MyCustomStoreSupplier("the-custom-store");
// Create CustomStoreBuilder for store name the-custom-store
MyCustomStoreBuilder customStoreBuilder = new MyCustomStoreBuilder("the-custom-store");
// Add the source topic
topology.addSource("input", "inputTopic");
// Add a custom processor that reads from the source topic
KafkaStreams streams = new KafkaStreams(topology, config);
streams.start();
@ -2478,7 +2492,7 @@ Note that in the <code>WordCountProcessor</code> implementation, users need to r
@@ -2478,7 +2492,7 @@ Note that in the <code>WordCountProcessor</code> implementation, users need to r
// This call to `count()` creates a state store named "word-count".
// The state store is discoverable and can be queried interactively.
KafkaStreams streams = new KafkaStreams(builder.build(), streamsConfiguration);
@ -2835,22 +2849,20 @@ Note that in the <code>WordCountProcessor</code> implementation, users need to r
@@ -2835,22 +2849,20 @@ Note that in the <code>WordCountProcessor</code> implementation, users need to r
<p>
For changelog topics you can also override the default configs on a per store basis.
This can be done by using any method overload that has a <code>StateStoreSupplier</code> as a parameter:
This can be done by using any method overload that has a <code>Materialized</code> as a parameter:
</p>
<preclass="brush: java;">
// a map to add topic config
Map<String, String> topicConfig = new HashMap<>();
Map<String, String> topicConfig = new HashMap<>();