How to split a stream of events into substreams

Problem:

you have events in a single Kafka topic, and you want to split it so that the events are placed into subtopics.

Edit this page

Example use case:

Suppose that you have a Kafka topic representing appearances of an actor or actress in a film, with each event denoting the genre. In this tutorial, we'll write a program that splits the stream into substreams based on the genre. We'll have a topic for drama films, a topic for fantasy films, and a topic for everything else.

Code example:

Try it

1
Initialize the project

To get started, make a new directory anywhere you’d like for this project:

mkdir split-stream && cd split-stream

2
Get Confluent Platform

Next, create the following docker-compose.yml file to obtain Confluent Platform:

---
version: '2'

services:
  zookeeper:
    image: confluentinc/cp-zookeeper:5.3.0
    hostname: zookeeper
    container_name: zookeeper
    ports:
      - "2181:2181"
    environment:
      ZOOKEEPER_CLIENT_PORT: 2181
      ZOOKEEPER_TICK_TIME: 2000

  broker:
    image: confluentinc/cp-enterprise-kafka:5.3.0
    hostname: broker
    container_name: broker
    depends_on:
      - zookeeper
    ports:
      - "29092:29092"
    environment:
      KAFKA_BROKER_ID: 1
      KAFKA_ZOOKEEPER_CONNECT: 'zookeeper:2181'
      KAFKA_LISTENER_SECURITY_PROTOCOL_MAP: PLAINTEXT:PLAINTEXT,PLAINTEXT_HOST:PLAINTEXT
      KAFKA_ADVERTISED_LISTENERS: PLAINTEXT://broker:9092,PLAINTEXT_HOST://localhost:29092
      KAFKA_METRIC_REPORTERS: io.confluent.metrics.reporter.ConfluentMetricsReporter
      KAFKA_OFFSETS_TOPIC_REPLICATION_FACTOR: 1
      KAFKA_GROUP_INITIAL_REBALANCE_DELAY_MS: 0
      CONFLUENT_METRICS_REPORTER_BOOTSTRAP_SERVERS: broker:9092
      CONFLUENT_METRICS_REPORTER_ZOOKEEPER_CONNECT: zookeeper:2181
      CONFLUENT_METRICS_REPORTER_TOPIC_REPLICAS: 1
      CONFLUENT_METRICS_ENABLE: 'true'
      CONFLUENT_SUPPORT_CUSTOMER_ID: 'anonymous'

  schema-registry:
    image: confluentinc/cp-schema-registry:5.3.0
    hostname: schema-registry
    container_name: schema-registry
    depends_on:
      - zookeeper
      - broker
    ports:
      - "8081:8081"
    environment:
      SCHEMA_REGISTRY_HOST_NAME: schema-registry
      SCHEMA_REGISTRY_KAFKASTORE_CONNECTION_URL: 'zookeeper:2181'
      SCHEMA_REGISTRY_LOG4J_ROOT_LOGLEVEL: WARN

And launch it by running:

docker-compose up

3
Configure the project

Then create the following Gradle build file, named build.gradle for the project:

buildscript {
    repositories {
        jcenter()
    }
    dependencies {
        classpath "com.commercehub.gradle.plugin:gradle-avro-plugin:0.15.1"
        classpath "com.github.jengelman.gradle.plugins:shadow:4.0.2"
    }
}

plugins {
    id "java"
    id "com.google.cloud.tools.jib" version "1.1.1"
}

sourceCompatibility = "1.8"
targetCompatibility = "1.8"
version = "0.0.1"

repositories {
    mavenCentral()
    jcenter()

    maven {
        url "http://packages.confluent.io/maven"
    }
}

apply plugin: "com.commercehub.gradle.plugin.avro"
apply plugin: "com.github.johnrengelman.shadow"

dependencies {
    compile "org.apache.avro:avro:1.8.2"
    implementation "org.slf4j:slf4j-simple:1.7.26"
    implementation "org.apache.kafka:kafka-streams:2.2.0"
    implementation "io.confluent:kafka-streams-avro-serde:5.2.0"
    testCompile "org.apache.kafka:kafka-streams-test-utils:2.2.0"
    testCompile "junit:junit:4.12"
}

test {
    testLogging {
        outputs.upToDateWhen { false }
        showStandardStreams = true
        exceptionFormat = "full"
    }
}

jar {
  manifest {
    attributes(
      "Class-Path": configurations.compile.collect { it.getName() }.join(" "),
      "Main-Class": "io.confluent.developer.SplitStream"
    )
  }
}

shadowJar {
    archiveName = "kstreams-split-standalone-${version}.${extension}"
}

And be sure to run the following command to obtain the Gradle wrapper:

gradle wrapper

Next, create a directory for configuration data:

mkdir configuration

Then create a development file at configuration/dev.properties:

application.id=splitting-app
bootstrap.servers=127.0.0.1:29092
schema.registry.url=http://127.0.0.1:8081

input.topic.name=acting-events
input.topic.partitions=1
input.topic.replication.factor=1

output.drama.topic.name=drama-acting-events
output.drama.topic.partitions=1
output.drama.topic.replication.factor=1

output.fantasy.topic.name=fantasy-acting-events
output.fantasy.topic.partitions=1
output.fantasy.topic.replication.factor=1

output.other.topic.name=other-acting-events
output.other.topic.partitions=1
output.other.topic.replication.factor=1

4
Create a schema for the events

Create a directory for the schemas that represent the events in the stream:

mkdir -p src/main/avro

Then create the following Avro schema file at src/main/avro/acting_event.avsc for the acting appearance events:

{
  "namespace": "io.confluent.developer.avro",
  "type": "record",
  "name": "ActingEvent",
  "fields": [
    {"name": "name", "type": "string"},
    {"name": "title", "type": "string"},
    {"name": "genre", "type": "string"}
  ]
}

Because we will use this Avro schema in our Java code, we’ll need to compile it. Run the following:

./gradlew build

5
Create the Kafka Streams topology

Create a directory for the Java files in this project:

mkdir -p src/main/java/io/confluent/developer

Then create the following file at src/main/java/io/confluent/developer/SplitStream.java. Notice the buildTopology method, which uses the Kafka Streams DSL. The branch method takes varargs of predicate functions over each record’s key and value. branch returns an array of KStream, with the order of the array matching the the ordering of the predicates. If no predicates are matched, the event gets dropped from further processing. Notice the last predicate, which simply returns true. This acts as an "else" statement to catch all events that don’t match the other predicates.

package io.confluent.developer;

import io.confluent.developer.avro.ActingEvent;
import io.confluent.kafka.serializers.AbstractKafkaAvroSerDeConfig;
import io.confluent.kafka.streams.serdes.avro.SpecificAvroSerde;
import org.apache.kafka.clients.admin.AdminClient;
import org.apache.kafka.clients.admin.NewTopic;
import org.apache.kafka.common.serialization.Serdes;
import org.apache.kafka.streams.KafkaStreams;
import org.apache.kafka.streams.StreamsBuilder;
import org.apache.kafka.streams.StreamsConfig;
import org.apache.kafka.streams.Topology;
import org.apache.kafka.streams.kstream.KStream;

import java.io.FileInputStream;
import java.io.IOException;
import java.util.*;
import java.util.concurrent.CountDownLatch;

public class SplitStream {

    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(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.String().getClass());
        props.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, SpecificAvroSerde.class);
        props.put(AbstractKafkaAvroSerDeConfig.SCHEMA_REGISTRY_URL_CONFIG, envProps.getProperty("schema.registry.url"));

        return props;
    }

    public Topology buildTopology(Properties envProps) {
        final StreamsBuilder builder = new StreamsBuilder();
        final String inputTopic = envProps.getProperty("input.topic.name");

        KStream<String, ActingEvent>[] branches = builder.<String, ActingEvent>stream(inputTopic)
                .branch((key, appearance) -> "drama".equals(appearance.getGenre()),
                        (key, appearance) -> "fantasy".equals(appearance.getGenre()),
                        (key, appearance) -> true);

        branches[0].to(envProps.getProperty("output.drama.topic.name"));
        branches[1].to(envProps.getProperty("output.fantasy.topic.name"));
        branches[2].to(envProps.getProperty("output.other.topic.name"));

        return builder.build();
    }

    public void createTopics(Properties envProps) {
        Map<String, Object> config = new HashMap<>();
        config.put("bootstrap.servers", envProps.getProperty("bootstrap.servers"));
        AdminClient client = AdminClient.create(config);

        List<NewTopic> topics = new ArrayList<>();

        topics.add(new NewTopic(
                envProps.getProperty("input.topic.name"),
                Integer.parseInt(envProps.getProperty("input.topic.partitions")),
                Short.parseShort(envProps.getProperty("input.topic.replication.factor"))));

        topics.add(new NewTopic(
                envProps.getProperty("output.drama.topic.name"),
                Integer.parseInt(envProps.getProperty("output.drama.topic.partitions")),
                Short.parseShort(envProps.getProperty("output.drama.topic.replication.factor"))));

        topics.add(new NewTopic(
                envProps.getProperty("output.fantasy.topic.name"),
                Integer.parseInt(envProps.getProperty("output.fantasy.topic.partitions")),
                Short.parseShort(envProps.getProperty("output.fantasy.topic.replication.factor"))));

        topics.add(new NewTopic(
                envProps.getProperty("output.other.topic.name"),
                Integer.parseInt(envProps.getProperty("output.other.topic.partitions")),
                Short.parseShort(envProps.getProperty("output.other.topic.replication.factor"))));

        client.createTopics(topics);
        client.close();
    }

    public Properties loadEnvProperties(String fileName) throws IOException {
        Properties envProps = new Properties();
        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.");
        }

        SplitStream ss = new SplitStream();
        Properties envProps = ss.loadEnvProperties(args[0]);
        Properties streamProps = ss.buildStreamsProperties(envProps);
        Topology topology = ss.buildTopology(envProps);

        ss.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();
                latch.countDown();
            }
        });

        try {
            streams.start();
            latch.await();
        } catch (Throwable e) {
            System.exit(1);
        }
        System.exit(0);
    }
}

6
Compile and run the Kafka Streams program

In your terminal, run:

./gradlew shadowJar

Now that an uberjar for the Kafka Streams application has been built, you can launch it locally. When you run the following, the prompt won’t return, because the application will run until you exit it:

java -jar build/libs/kstreams-split-standalone-0.0.1.jar configuration/dev.properties

7
Produce events to the input topic

In a new terminal, run:

docker exec -i schema-registry /usr/bin/kafka-avro-console-producer --topic acting-events --broker-list broker:9092 --property value.schema="$(< src/main/avro/acting_event.avsc)"

When the console producer starts, it will log some messages and hang, waiting for your input. Type in one line at a time and press enter to send it. Each line represents an event. To send all of the events below, paste the following into the prompt and press enter:

{"name": "Meryl Streep", "title": "The Iron Lady", "genre": "drama"}
{"name": "Will Smith", "title": "Men in Black", "genre": "comedy"}
{"name": "Matt Damon", "title": "The Martian", "genre": "drama"}
{"name": "Judy Garland", "title": "The Wizard of Oz", "genre": "fantasy"}
{"name": "Jennifer Aniston", "title": "Office Space", "genre": "comedy"}
{"name": "Bill Murray", "title": "Ghostbusters", "genre": "fantasy"}
{"name": "Christian Bale", "title": "The Dark Knight", "genre": "crime"}
{"name": "Laura Dern", "title": "Jurassic Park", "genre": "fantasy"}
{"name": "Keanu Reeves", "title": "The Matrix", "genre": "fantasy"}
{"name": "Russell Crowe", "title": "Gladiator", "genre": "drama"}
{"name": "Diane Keaton", "title": "The Godfather: Part II", "genre": "crime"}

8
Consume the event subsets from the output topics

Leave your original terminal running. To consume the output events from each of the topic, you’ll need to open several new terminal windows. In each instance, the prompt will hang, waiting for more events to arrive. To continue studying the example, send more events through the input terminal prompt. Otherwise, you can Control-C to exit the process.

First, to consume the events of drama films, run the following:

docker exec -it schema-registry /usr/bin/kafka-avro-console-consumer --topic drama-acting-events --bootstrap-server broker:9092 --from-beginning

This should yield the following messages:

{"name":"Meryl Streep","title":"The Iron Lady","genre":"drama"}
{"name":"Matt Damon","title":"The Martian","genre":"drama"}
{"name":"Russell Crowe","title":"Gladiator","genre":"drama"}

Second, to consume those from fantasy films, run the following:

docker exec -it schema-registry /usr/bin/kafka-avro-console-consumer --topic fantasy-acting-events --bootstrap-server broker:9092 --from-beginning

This should yield the following messages:

{"name":"Judy Garland","title":"The Wizard of Oz","genre":"fantasy"}
{"name":"Bill Murray","title":"Ghostbusters","genre":"fantasy"}
{"name":"Laura Dern","title":"Jurassic Park","genre":"fantasy"}
{"name":"Keanu Reeves","title":"The Matrix","genre":"fantasy"}

And finally, to consume all the other genres, run the following:

docker exec -it schema-registry /usr/bin/kafka-avro-console-consumer --topic other-acting-events --bootstrap-server broker:9092 --from-beginning

This should yield the following messages:

{"name":"Will Smith","title":"Men in Black","genre":"comedy"}
{"name":"Jennifer Aniston","title":"Office Space","genre":"comedy"}
{"name":"Christian Bale","title":"The Dark Knight","genre":"crime"}
{"name":"Diane Keaton","title":"The Godfather: Part II","genre":"crime"}

Test it

1
Create a test configuration file

First, create a test file at configuration/test.properties:

application.id=splitting-app
bootstrap.servers=127.0.0.1:29092
schema.registry.url=http://127.0.0.1:8081

input.topic.name=acting-events
input.topic.partitions=1
input.topic.replication.factor=1

output.drama.topic.name=drama-acting-events
output.drama.topic.partitions=1
output.drama.topic.replication.factor=1

output.fantasy.topic.name=fantasy-acting-events
output.fantasy.topic.partitions=1
output.fantasy.topic.replication.factor=1

output.other.topic.name=other-acting-events
output.other.topic.partitions=1
output.other.topic.replication.factor=1

2
Write a test

Then, create a directory for the tests to live in:

mkdir -p src/test/java/io/confluent/developer

Create the following test file at src/test/java/io/confluent/developer/SplitStreamTest.java:

package io.confluent.developer;

import io.confluent.developer.avro.ActingEvent;
import io.confluent.kafka.streams.serdes.avro.SpecificAvroDeserializer;
import io.confluent.kafka.streams.serdes.avro.SpecificAvroSerializer;
import org.apache.kafka.clients.consumer.ConsumerRecord;
import org.apache.kafka.clients.producer.ProducerRecord;
import org.apache.kafka.common.header.internals.RecordHeaders;
import org.apache.kafka.common.serialization.Deserializer;
import org.apache.kafka.common.serialization.Serdes;
import org.apache.kafka.common.serialization.Serializer;
import org.apache.kafka.streams.Topology;
import org.apache.kafka.streams.TopologyTestDriver;
import org.apache.kafka.streams.test.ConsumerRecordFactory;
import org.junit.Assert;
import org.junit.Test;

import java.io.IOException;
import java.util.*;

public class SplitStreamTest {

    private final static String TEST_CONFIG_FILE = "configuration/test.properties";

    public SpecificAvroSerializer<ActingEvent> makeSerializer(Properties envProps) {
        SpecificAvroSerializer<ActingEvent> serializer = new SpecificAvroSerializer<>();

        Map<String, String> config = new HashMap<>();
        config.put("schema.registry.url", envProps.getProperty("schema.registry.url"));
        serializer.configure(config, false);

        return serializer;
    }

    public SpecificAvroDeserializer<ActingEvent> makeDeserializer(Properties envProps) {
        SpecificAvroDeserializer<ActingEvent> deserializer = new SpecificAvroDeserializer<>();

        Map<String, String> config = new HashMap<>();
        config.put("schema.registry.url", envProps.getProperty("schema.registry.url"));
        deserializer.configure(config, false);

        return deserializer;
    }

    private List<ActingEvent> readOutputTopic(TopologyTestDriver testDriver,
                                              String topic,
                                              Deserializer<String> keyDeserializer,
                                              SpecificAvroDeserializer<ActingEvent> valueDeserializer) {
        List<ActingEvent> results = new ArrayList<>();

        while (true) {
            ProducerRecord<String, ActingEvent> record = testDriver.readOutput(topic, keyDeserializer, valueDeserializer);

            if (record != null) {
                results.add(record.value());
            } else {
                break;
            }
        }

        return results;
    }

    @Test
    public void testSplitStream() throws IOException {
        SplitStream ss = new SplitStream();
        Properties envProps = ss.loadEnvProperties(TEST_CONFIG_FILE);
        Properties streamProps = ss.buildStreamsProperties(envProps);

        String inputTopic = envProps.getProperty("input.topic.name");
        String outputDramaTopic = envProps.getProperty("output.drama.topic.name");
        String outputFantasyTopic = envProps.getProperty("output.fantasy.topic.name");
        String outputOtherTopic = envProps.getProperty("output.other.topic.name");

        Topology topology = ss.buildTopology(envProps);
        TopologyTestDriver testDriver = new TopologyTestDriver(topology, streamProps);

        Serializer<String> keySerializer = Serdes.String().serializer();
        SpecificAvroSerializer<ActingEvent> valueSerializer = makeSerializer(envProps);

        Deserializer<String> keyDeserializer = Serdes.String().deserializer();
        SpecificAvroDeserializer<ActingEvent> valueDeserializer = makeDeserializer(envProps);

        ConsumerRecordFactory<String, ActingEvent> inputFactory = new ConsumerRecordFactory<>(keySerializer, valueSerializer);

        ActingEvent streep = ActingEvent.newBuilder()
                .setName("Meryl Streep").setTitle("The Iron Lady").setGenre("drama").build();
        ActingEvent smith = ActingEvent.newBuilder()
                .setName("Will Smith").setTitle("Men in Black").setGenre("comedy").build();
        ActingEvent damon = ActingEvent.newBuilder()
                .setName("Matt Damon").setTitle("The Martian").setGenre("drama").build();
        ActingEvent garland = ActingEvent.newBuilder()
                .setName("Judy Garland").setTitle("The Wizard of Oz").setGenre("fantasy").build();
        ActingEvent aniston = ActingEvent.newBuilder()
                .setName("Jennifer Aniston").setTitle("Office Space").setGenre("comedy").build();
        ActingEvent murray = ActingEvent.newBuilder()
                .setName("Bill Murray").setTitle("Ghostbusters").setGenre("fantasy").build();
        ActingEvent bale = ActingEvent.newBuilder()
                .setName("Christian Bale").setTitle("The Dark Knight").setGenre("crime").build();
        ActingEvent dern = ActingEvent.newBuilder()
                .setName("Laura Dern").setTitle("Jurassic Park").setGenre("fantasy").build();
        ActingEvent reeves = ActingEvent.newBuilder()
                .setName("Keanu Reeves").setTitle("The Matrix").setGenre("fantasy").build();
        ActingEvent crowe = ActingEvent.newBuilder()
                .setName("Russell Crowe").setTitle("Gladiator").setGenre("drama").build();
        ActingEvent keaton = ActingEvent.newBuilder()
                .setName("Diane Keaton").setTitle("The Godfather: Part II").setGenre("crime").build();

        List<ActingEvent> input = new ArrayList<>();
        input.add(streep);
        input.add(smith);
        input.add(damon);
        input.add(garland);
        input.add(aniston);
        input.add(murray);
        input.add(bale);
        input.add(dern);
        input.add(reeves);
        input.add(crowe);
        input.add(keaton);

        List<ActingEvent> expectedDrama = new ArrayList<>();
        expectedDrama.add(streep);
        expectedDrama.add(damon);
        expectedDrama.add(crowe);

        List<ActingEvent> expectedFantasy = new ArrayList<>();
        expectedFantasy.add(garland);
        expectedFantasy.add(murray);
        expectedFantasy.add(dern);
        expectedFantasy.add(reeves);

        List<ActingEvent> expectedOther = new ArrayList<>();
        expectedOther.add(smith);
        expectedOther.add(aniston);
        expectedOther.add(bale);
        expectedOther.add(keaton);

        for (ActingEvent event : input) {
            testDriver.pipeInput(inputFactory.create(inputTopic, event.getName(), event));
        }

        List<ActingEvent> actualDrama = readOutputTopic(testDriver, outputDramaTopic, keyDeserializer, valueDeserializer);
        List<ActingEvent> actualFantasy = readOutputTopic(testDriver, outputFantasyTopic, keyDeserializer, valueDeserializer);
        List<ActingEvent> actualOther = readOutputTopic(testDriver, outputOtherTopic, keyDeserializer, valueDeserializer);

        Assert.assertEquals(expectedDrama, actualDrama);
        Assert.assertEquals(expectedFantasy, actualFantasy);
        Assert.assertEquals(expectedOther, actualOther);
    }

}

3
Invoke the tests

Now run the test, which is as simple as:

./gradlew test

Take it to production

1
Create a production configuration file

First, create a new configuration file at configuration/prod.properties with the following content. Be sure to fill in the addresses of your production hosts and change any other parameters that make sense for your setup.

application.id=splitting-app
bootstrap.servers=<< FILL ME IN >>
schema.registry.url=<< FILL ME IN >>

input.topic.name=acting-events
input.topic.partitions=<< FILL ME IN >>
input.topic.replication.factor=<< FILL ME IN >>

output.drama.topic.name=drama-acting-events
output.drama.topic.partitions=<< FILL ME IN >>
output.drama.topic.replication.factor=<< FILL ME IN >>

output.fantasy.topic.name=fantasy-acting-events
output.fantasy.topic.partitions=<< FILL ME IN >>
output.fantasy.topic.replication.factor=<< FILL ME IN >>

output.other.topic.name=other-acting-events
output.other.topic.partitions=<< FILL ME IN >>
output.other.topic.replication.factor=<< FILL ME IN >>

2
Build a Docker image

In your terminal, execute the following to invoke the Jib plugin to build an image:

gradle jibDockerBuild --image=io.confluent.developer/kstreams-split:0.0.1

3
Launch the container

Finally, launch the container using your preferred container orchestration service. If you want to run it locally, you can execute the following:

docker run -v $PWD/configuration/prod.properties:/config.properties io.confluent.developer/kstreams-split:0.0.1 config.properties