Create a directory for the Java files in this project:
mkdir -p src/main/java/io/confluent/developer
Before you create the Kafka Streams application file let’s go over the key points of the application. In this tutorial, instead of performing an operation on each key-value pair, you want to store the results in a state store and execute your business logic at regular intervals. In other words, you want to schedule an operation and Kafka Streams will run your code at regular intervals. In this case you’ll use the ProcessorContext.schedule method.
|
Since the KStream.transform method can potentially change the key, using this method flags the KStream instance as needing a repartition. But the repartition only happens if you perform a join or an aggregation after the transform. I used transform in this tutorial as it makes for a better example because you can use the ProcessorContext.forward method. Additionally, you’re not doing any joins or aggregations, so there’s no repartition required. But it’s important to consider your requirements and in most cases use a transformValues instead.
|
Now let’s take a look at some of the key points from the application.
For context your application consumes from a topic with information how long users have been logged into different applications. Your goal is to emit the user with the longest login times across all applications every five seconds. To do this you track the total login time per user in a state store. Additionally, every twenty seconds you want reset the cumulative times to zero every twenty seconds.
|
The following detailed sections are already included in the application file, we’re just taking a detailed step through the code before you create it.
|
Using the Transformer in the Kafka Streams application
final KStream<String, LoginTime> loginTimeStream = builder.stream(loginTimeInputTopic, Consumed.with(Serdes.String(), loginTimeSerde));
loginTimeStream.transform(getTransformerSupplier(loginTimeStore), Named.as("max-login-time-transformer"),loginTimeStore) (1)
.to(outputTopic, Produced.with(Serdes.String(), Serdes.Long()));
private TransformerSupplier<String, LoginTime, KeyValue<String, Long>> getTransformerSupplier(final String storeName) {
return () -> new Transformer<String, LoginTime, KeyValue<String, Long>>() { (2)
private KeyValueStore<String, Long> store;
private ProcessorContext context;
@Override
public void init(ProcessorContext context) { (3)
this.context = context;
store = (KeyValueStore<String, Long>) this.context.getStateStore(storeName);
this.context.schedule(Duration.ofSeconds(5), PunctuationType.STREAM_TIME, this::streamTimePunctuator); (4)
this.context.schedule(Duration.ofSeconds(20), PunctuationType.WALL_CLOCK_TIME, this::wallClockTimePunctuator); (5)
}
@Override
public KeyValue<String, Long> transform(String key, LoginTime value) { (6)
Long currentVT = store.putIfAbsent(key, value.getLogintime());
if (currentVT != null) {
store.put(key, currentVT + value.getLogintime());
}
return null;
}
1 |
Adding a transform operation to the KStream |
2 |
Using a lambda since the TransformerSupplier interface only has one method, get(). Calling get() should always return a new instance of a Transformer |
3 |
The init method used to configure the transformer. It’s in the init method you schedule any punctuations. Kafka Streams calls the init method for all processors/transformers. |
4 |
Scheduling a punctuation to occur based on STREAM_TIME every five seconds. The third parameter is a method handle used for the Punctuator interface. |
5 |
Scheduling a punctuation to fire based on WALL_CLOCK_TIME every twenty seconds. The third parameter is a method handle used for the Punctuator interface. |
6 |
The transform method. All you are doing here is incrementing the total time a user is logged in and storing it in a state store. |
From the above code section, you are adding a transform
operation to the stream reading from the input topic. The key parts of this section are points four and five where you schedule the punctuations. There are two schedule operations one using STREAM_TIME
and another using WALL_CLOCK_TIME
. These two are from the PunctuationType enum.
The stream-time punctuation fires based on timestamps on the incoming records, stream-time only advances as records arrive. The wall-clock-time punctuation fires based on system time advanced at the polling interval and is independent of the rate incoming messages. Read how Kafka Streams supports notions of time for more information.
Next let’s talk discuss the scheduling in a little more detail.
When you schedule a punctuation, you provide three parameters:
-
How often the punctuation should execute defined as a type of Duration
-
The PunctuationType
either stream-time or wall-clock tome
-
An instance of the Punctuator interface. Since the Punctuator
interface has only one , punctuate
, typically you’ll use either a lambda expression or a method reference. In this case we’ve used a method reference.
Now let’s take a look at these methods.
Method references used for punctuations
void wallClockTimePunctuator(Long timestamp){ (1)
try (KeyValueIterator<String, Long> iterator = store.all()) {
while (iterator.hasNext()) {
KeyValue<String, Long> keyValue = iterator.next();
store.put(keyValue.key, 0L);
}
}
System.out.println("@" + new Date(timestamp) +" Reset all view-times to zero");
}
void streamTimePunctuator(Long timestamp) { (2)
Long maxValue = Long.MIN_VALUE;
String maxValueKey = "";
try (KeyValueIterator<String, Long> iterator = store.all()) {
while (iterator.hasNext()) {
KeyValue<String, Long> keyValue = iterator.next();
if (keyValue.value > maxValue) {
maxValue = keyValue.value;
maxValueKey = keyValue.key;
}
}
}
context.forward(maxValueKey +" @" + new Date(timestamp), maxValue); (3)
}
1 |
The wallClockTimePunctuator resets the times for all users to zero every 20 seconds. |
2 |
The streamTimePunctuator calculates the user with the largest logged in time |
3 |
Forwarding the results, in this case to a topic |
That wraps up our discussion for the finer points of the code for this tutorial. Now create the following file at src/main/java/io/confluent/developer/KafkaStreamsPunctuation.java
package io.confluent.developer;
import io.confluent.developer.avro.LoginTime;
import io.confluent.kafka.serializers.AbstractKafkaSchemaSerDeConfig;
import io.confluent.kafka.streams.serdes.avro.SpecificAvroSerde;
import org.apache.avro.specific.SpecificRecord;
import org.apache.kafka.clients.admin.AdminClient;
import org.apache.kafka.clients.admin.NewTopic;
import org.apache.kafka.common.serialization.Serde;
import org.apache.kafka.common.serialization.Serdes;
import org.apache.kafka.streams.KafkaStreams;
import org.apache.kafka.streams.KeyValue;
import org.apache.kafka.streams.StreamsBuilder;
import org.apache.kafka.streams.StreamsConfig;
import org.apache.kafka.streams.Topology;
import org.apache.kafka.streams.kstream.Consumed;
import org.apache.kafka.streams.kstream.KStream;
import org.apache.kafka.streams.kstream.Named;
import org.apache.kafka.streams.kstream.Produced;
import org.apache.kafka.streams.kstream.Transformer;
import org.apache.kafka.streams.kstream.TransformerSupplier;
import org.apache.kafka.streams.processor.ProcessorContext;
import org.apache.kafka.streams.processor.PunctuationType;
import org.apache.kafka.streams.state.KeyValueIterator;
import org.apache.kafka.streams.state.KeyValueStore;
import org.apache.kafka.streams.state.StoreBuilder;
import org.apache.kafka.streams.state.Stores;
import java.io.FileInputStream;
import java.io.IOException;
import java.time.Duration;
import java.util.ArrayList;
import java.util.Date;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Properties;
import java.util.concurrent.CountDownLatch;
public class KafkaStreamsPunctuation {
public Properties buildStreamsProperties(Properties envProps) {
Properties props = new Properties();
props.put(StreamsConfig.APPLICATION_ID_CONFIG, envProps.getProperty("application.id"));
props.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, envProps.getProperty("bootstrap.servers"));
props.put(AbstractKafkaSchemaSerDeConfig.SCHEMA_REGISTRY_URL_CONFIG, envProps.getProperty("schema.registry.url"));
return props;
}
public Topology buildTopology(Properties envProps) {
final StreamsBuilder builder = new StreamsBuilder();
final String loginTimeInputTopic = envProps.getProperty("input.topic.name");
final String outputTopic = envProps.getProperty("output.topic.name");
final String loginTimeStore = "logintime-store";
final Serde<LoginTime> loginTimeSerde = getSpecificAvroSerde(envProps);
StoreBuilder<KeyValueStore<String, Long>> storeBuilder = Stores.keyValueStoreBuilder(Stores.inMemoryKeyValueStore(loginTimeStore),Serdes.String(), Serdes.Long());
builder.addStateStore(storeBuilder);
final KStream<String, LoginTime> loginTimeStream = builder.stream(loginTimeInputTopic, Consumed.with(Serdes.String(), loginTimeSerde));
loginTimeStream.transform(getTransformerSupplier(loginTimeStore), Named.as("max-login-time-transformer"),loginTimeStore)
.to(outputTopic, Produced.with(Serdes.String(), Serdes.Long()));
return builder.build();
}
private TransformerSupplier<String, LoginTime, KeyValue<String, Long>> getTransformerSupplier(final String storeName) {
return () -> new Transformer<String, LoginTime, KeyValue<String, Long>>() {
private KeyValueStore<String, Long> store;
private ProcessorContext context;
@Override
public void init(ProcessorContext context) {
this.context = context;
store = (KeyValueStore<String, Long>) this.context.getStateStore(storeName);
this.context.schedule(Duration.ofSeconds(5), PunctuationType.STREAM_TIME, this::streamTimePunctuator);
this.context.schedule(Duration.ofSeconds(20), PunctuationType.WALL_CLOCK_TIME, this::wallClockTimePunctuator);
}
void wallClockTimePunctuator(Long timestamp){
try (KeyValueIterator<String, Long> iterator = store.all()) {
while (iterator.hasNext()) {
KeyValue<String, Long> keyValue = iterator.next();
store.put(keyValue.key, 0L);
}
}
System.out.println("@" + new Date(timestamp) +" Reset all view-times to zero");
}
void streamTimePunctuator(Long timestamp) {
Long maxValue = Long.MIN_VALUE;
String maxValueKey = "";
try (KeyValueIterator<String, Long> iterator = store.all()) {
while (iterator.hasNext()) {
KeyValue<String, Long> keyValue = iterator.next();
if (keyValue.value > maxValue) {
maxValue = keyValue.value;
maxValueKey = keyValue.key;
}
}
}
context.forward(maxValueKey +" @" + new Date(timestamp), maxValue);
}
@Override
public KeyValue<String, Long> transform(String key, LoginTime value) {
Long currentVT = store.putIfAbsent(key, value.getLogintime());
if (currentVT != null) {
store.put(key, currentVT + value.getLogintime());
}
return null;
}
@Override
public void close() {
}
};
}
static <T extends SpecificRecord> SpecificAvroSerde<T> getSpecificAvroSerde(final Properties envProps) {
final SpecificAvroSerde<T> specificAvroSerde = new SpecificAvroSerde<>();
final HashMap<String, String> serdeConfig = new HashMap<>();
serdeConfig.put(AbstractKafkaSchemaSerDeConfig.SCHEMA_REGISTRY_URL_CONFIG,
envProps.getProperty("schema.registry.url"));
specificAvroSerde.configure(serdeConfig, false);
return specificAvroSerde;
}
public void createTopics(final Properties envProps) {
final Map<String, Object> config = new HashMap<>();
config.put("bootstrap.servers", envProps.getProperty("bootstrap.servers"));
try (final AdminClient client = AdminClient.create(config)) {
final List<NewTopic> topics = new ArrayList<>();
topics.add(new NewTopic(
envProps.getProperty("output.topic.name"),
Integer.parseInt(envProps.getProperty("output.topic.partitions")),
Short.parseShort(envProps.getProperty("output.topic.replication.factor"))));
client.createTopics(topics);
}
}
public Properties loadEnvProperties(String fileName) throws IOException {
final Properties envProps = new Properties();
final FileInputStream input = new FileInputStream(fileName);
envProps.load(input);
input.close();
return envProps;
}
public static void main(String[] args) throws Exception {
if (args.length < 1) {
throw new IllegalArgumentException("This program takes one argument: the path to an environment configuration file.");
}
final KafkaStreamsPunctuation instance = new KafkaStreamsPunctuation();
final Properties envProps = instance.loadEnvProperties(args[0]);
final Properties streamProps = instance.buildStreamsProperties(envProps);
final Topology topology = instance.buildTopology(envProps);
instance.createTopics(envProps);
final KafkaStreams streams = new KafkaStreams(topology, streamProps);
final CountDownLatch latch = new CountDownLatch(1);
// Attach shutdown handler to catch Control-C.
Runtime.getRuntime().addShutdownHook(new Thread("streams-shutdown-hook") {
@Override
public void run() {
streams.close(Duration.ofSeconds(5));
latch.countDown();
}
});
try {
streams.start();
latch.await();
} catch (Throwable e) {
System.exit(1);
}
System.exit(0);
}
}