The Hitchhiker’s Guide to Stream Processing

In my previous post I discussed some of the features of different stream processing platforms, but I did not touch on how one goes about choosing one.  In the book Kafka: The Definitive Guide, the authors provide some valuable advice on how to make a choice.  Below I present three additional tips for choosing a stream processing platform.

  1. Favor reuse over polyglot streaming.
  2. Try the Kafka abstraction funnel.
  3. Check out the Stream Processing Kickstarter project.

Favor Reuse Over Polyglot Streaming

One of the characteristics of many microservice architectures is the use of several different data stores, which is often referred to as polyglot persistence.  The idea is that each microservice should be responsible for its own data, which often leads to each microservice possibly introducing a different data store into the organization.   However, at my last company, Yammer, this led to an operational nightmare as we attempted to manage and develop expertise in a variety of data stores.  In the end, we decided to consolidate on one relational database (PostgreSQL) and one non-relational database (HBase).

The real underlying goal for data use in microservices is to eliminate coupling between business subdomains in an application, which can be better achieved by schema independence, rather than polyglot persistence.  Schema independence can easily be achieved by partitioning data properly within a single data store, using namespaces, schemas, or other techniques, and does not necessitate the use of polyglot persistence.

In a streaming architecture, one of the goals is to share data rather than to keep it siloed.  This is often referred to as turning the database inside-out, as achieved by many event-driven systems. This means that schemas need to be shared as well, which leads to the use of schema registries.  Consequently, it makes even less sense to use more than one streaming platform in an organization, which would lead to polyglot streaming.   Therefore, if your organization is already using a stream processing platform, perhaps in a different group from the one that you are in, try using that.

Try the Kafka Abstraction Funnel

Kafka has emerged as the foundation for stream processing in today’s enterprises.  Kafka provides a low-level Producer-Consumer API, but it also comes bundled with a stream processing framework called Kafka Streams, which has both a Processor API and a Streams DSL (that is built on top of the Processor API).  Additionally, Confluent (the company I work for) has just released KSQL, which is built on top of Kafka Streams.

These four layers (KSQL, the Kafka Streams DSL, the Kafka Streams Processor API, and the Producer-Consumer API) form a hierarchy of abstractions, which I refer to as the Kafka abstraction funnel.   In accordance with the 80-20 rule, the top of the funnel, KSQL, should be able to handle 80% of your use cases, and for the remaining 20%, you can drop down into the Kafka Streams DSL.  Likewise, the Kafka Streams DSL should be able to handle 80% of the use cases that KSQL cannot handle, and for the remaining 20%, you can drop down into the Kafka Streams Processor API.  Finally, at the bottom of the funnel is the Producer-Consumer API, which should be able to handle everything else.

The Kafka abstraction funnel is very easy to get started with, provides support for stream-relational processing, and simply builds on technologies already provided with Kafka.  Give it a try if you can.

I’ll have more to say about the power of the Kafka abstraction funnel in my next post.

Check Out the Stream Processing Kickstarter Project

Martin Kleppman has likened Kafka to Unix pipes, with the stream processor being the Unix program that reads from one pipe and writes to another.  However, if you’re using Kafka but you’ve not settled on the Kafka abstraction funnel, figuring out exactly which stream processor to use can be a bewildering task.

There are other posts that compare the different streaming platforms.  Often these comparisons show how to write a WordCount application, which has become the “Hello World” example of data processing.   However, most of the WordCount examples don’t actually show how to integrate with Kafka.  Therefore I’ve put together the Stream Processing Kickstarter that has the WordCount application written in each of the most popular streaming frameworks.  Furthermore, in each WordCount application, I show how to read textual data from a Kafka topic and then output the word counts to another Kafka topic.  Also, each example in the project can be easily executed against an embedded Kafka cluster by simply running a unit test.

Below I show the code for each WordCount application so you can get a sense what it is like to use each streaming platform.1

Beam

In Apache Beam, integration with Kafka is achieved by using a Kafka reader transform and a Kafka writer transform.

public class BeamWordCount implements WordCount {

  @Override
  public void countWords(
      String bootstrapServers, String zookeeperConnect,
      String inputTopic, String outputTopic) {

    PipelineOptions options = PipelineOptionsFactory.create();
    Pipeline p = Pipeline.create(options);

    KafkaIO.Read<String, String> reader = KafkaIO.<String, String>read()
        .withBootstrapServers(bootstrapServers)
        .withTopic(inputTopic)
        .withKeyDeserializer(StringDeserializer.class)
        .withValueDeserializer(StringDeserializer.class)
        .updateConsumerProperties(
            ImmutableMap.of(
                ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest"));

    PCollection<String> input = p.apply(reader.withoutMetadata())
        .apply(MapElements
            .into(TypeDescriptors.strings())
            .via(KV::getValue))
        .apply(ParDo.of(new AddTimestampFn()));

    PCollection<String> windowedWords = input.apply(
        Window.<String>into(FixedWindows.of(Duration.standardSeconds(1)))
            .triggering(Repeatedly.forever(AfterPane.elementCountAtLeast(1)))
            .withAllowedLateness(Duration.ZERO)
            .accumulatingFiredPanes());

    PCollection<KV<String, Long>> wordCounts =
        windowedWords.apply(new CountWords());

    wordCounts.apply(KafkaIO.<String, Long>write()
        .withBootstrapServers(bootstrapServers)
        .withTopic(outputTopic)
        .withKeySerializer(StringSerializer.class)
        .withValueSerializer(LongSerializer.class));

    p.run().waitUntilFinish(Duration.standardSeconds(5));
  }

  @Override
  public void close() {
  }

  static class AddTimestampFn extends DoFn<String, String> {

    @ProcessElement
    public void processElement(ProcessContext c) {
      c.outputWithTimestamp(c.element(), Instant.now());
    }
  }

  static class ExtractWordsFn extends DoFn<String, String> {

    @ProcessElement
    public void processElement(ProcessContext c) {
      // Split the line into words.
      String[] words = c.element().toLowerCase().split("\\W+");

      // Output each word encountered into the output PCollection.
      for (String word : words) {
        if (!word.isEmpty()) {
          c.output(word);
        }
      }
    }
  }

  static class CountWords extends PTransform<PCollection<String>,
      PCollection<KV<String, Long>>> {
    @Override
    public PCollection<KV<String, Long>> expand(PCollection<String> lines) {

      // Convert lines of text into individual words.
      PCollection<String> words = lines.apply(
          ParDo.of(new ExtractWordsFn()));

      // Count the number of times each word occurs.
      PCollection<KV<String, Long>> wordCounts =
          words.apply(Count.perElement());

      return wordCounts;
    }
  }
}

Flink

The Flink example uses a Kafka source and a Kafka sink.

public class FlinkWordCount implements WordCount {

  @Override
  public void countWords(
      String bootstrapServers, String zookeeperConnect,
      String inputTopic, String outputTopic) throws Exception {

    Properties props = new Properties();
    props.put("bootstrap.servers", bootstrapServers);
    props.put("group.id", "testgroup");
    props.put("auto.offset.reset", "earliest");

    // set up the execution environment
    final StreamExecutionEnvironment env =
        StreamExecutionEnvironment.getExecutionEnvironment();

    DataStream<String> input = env
        .addSource(new FlinkKafkaConsumer011<>(
            inputTopic,
            new SimpleStringSchema(),
            props
        ));

    DataStream<Tuple2<String, Long>> counts = input
        .map(line -> line.toLowerCase().split("\\W+"))
        .flatMap((String[] tokens, Collector<Tuple2<String, Long>> out) -> {
          // emit the pairs with non-zero-length words
          Arrays.stream(tokens)
              .filter(t -> t.length() > 0)
              .forEach(t -> out.collect(new Tuple2<>(t, 1L)));
        })
        .keyBy(0)
        .sum(1);

    counts.addSink(
        new FlinkKafkaProducer011<>(
            outputTopic,
            new WordCountSerializer(outputTopic),
            props
        ));

    // execute program
    env.execute("Streaming WordCount Example");
  }

  @Override
  public void close() {
  }

  static class WordCountSerializer implements
      KeyedSerializationSchema<Tuple2<String, Long>>,
      java.io.Serializable {

    private final String outputTopic;

    public WordCountSerializer(String outputTopic) {
      this.outputTopic = outputTopic;
    }

    public byte[] serializeKey(Tuple2<String, Long> element) {
      return new StringSerializer().serialize(null, element.getField(0));
    }

    public byte[] serializeValue(Tuple2<String, Long> element) {
      return new LongSerializer().serialize(null, element.getField(1));
    }

    public String getTargetTopic(Tuple2<String, Long> element) {
      return outputTopic;
    }
  }
}

Kafka Streams

Kafka Streams, being built directly atop Kafka, has the simplest integration with Kafka.  An input stream is constructed directly from a Kafka topic and then later the data in the stream is sent directly to another Kafka topic.

public class KafkaStreamsWordCount implements WordCount {

  private KafkaStreams streams;

  @Override
  public void countWords(
      String bootstrapServers, String zookeeperConnect,
      String inputTopic, String outputTopic) {

    Properties props = new Properties();
    props.put(StreamsConfig.APPLICATION_ID_CONFIG, "wordcount");
    props.put(StreamsConfig.CLIENT_ID_CONFIG, "wordcount-example-client");
    props.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, bootstrapServers);
    props.put(
        StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG,
        Serdes.String().getClass().getName()
    );
    props.put(
        StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG,
        Serdes.String().getClass().getName()
    );
    props.put(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, 0);
    props.put(
        StreamsConfig.STATE_DIR_CONFIG,
        Utils.tempDirectory().getAbsolutePath()
    );

    StreamsBuilder builder = new StreamsBuilder();

    KStream<String, String> input = builder.stream(inputTopic);

    KTable<String, Long> wordCounts = input
        .flatMapValues(
            value -> Arrays.asList(value.toLowerCase().split("\\W+")))
        .groupBy((key, word) -> word)
        .count();

    wordCounts.toStream().to(
        outputTopic, Produced.with(Serdes.String(), Serdes.Long()));

    streams = new KafkaStreams(builder.build(), props);
    streams.start();
  }

  @Override
  public void close() {
    streams.close();
  }
}

Samza

Samza can also create streams directly from Kafka topics by using a Kafka system factory.

public class SamzaWordCount implements WordCount {

  @Override
  public void countWords(
      String bootstrapServers, String zookeeperConnect,
      String inputTopic, String outputTopic) {

    Map<String, String> configs = new HashMap<>();

    configs.put(JobConfig.JOB_NAME(), "word-count");
    configs.put(JobConfig.PROCESSOR_ID(), "1");
    configs.put(
        JobCoordinatorConfig.JOB_COORDINATION_UTILS_FACTORY,
        PassthroughCoordinationUtilsFactory.class.getName()
    );
    configs.put(
        JobCoordinatorConfig.JOB_COORDINATOR_FACTORY,
        PassthroughJobCoordinatorFactory.class.getName()
    );
    configs.put(
        TaskConfig.GROUPER_FACTORY(),
        SingleContainerGrouperFactory.class.getName()
    );
    configs.put(
        "systems.kafka.samza.factory",
        "org.apache.samza.system.kafka.KafkaSystemFactory"
    );
    configs.put("systems.kafka.producer.bootstrap.servers", bootstrapServers);
    configs.put("systems.kafka.consumer.zookeeper.connect", zookeeperConnect);
    configs.put("systems.kafka.samza.offset.default", "oldest");
    configs.put("systems.kafka.default.stream.replication.factor", "1");
    configs.put("job.default.system", "kafka");

    LocalApplicationRunner runner =
        new LocalApplicationRunner(new MapConfig(configs));
    runner.run(new WordCountApplication(inputTopic, outputTopic));
    runner.waitForFinish();
  }

  @Override
  public void close() {
  }

  static class WordCountApplication implements StreamApplication {

    private String inputTopic;
    private String outputTopic;

    public WordCountApplication(String inputTopic, String outputTopic) {
      this.inputTopic = inputTopic;
      this.outputTopic = outputTopic;
    }

    @Override
    public void init(StreamGraph graph, Config config) {
      MessageStream<KV<String, String>> words =
          graph.getInputStream(
              inputTopic, KVSerde.of(new StringSerde(), new StringSerde()));
      OutputStream<KV<String, Long>> counts =
          graph.getOutputStream(
              outputTopic, KVSerde.of(new StringSerde(), new LongSerde()));

      words.flatMap(kv -> Arrays.asList(kv.value.toLowerCase().split("\\W+")))
          .window(Windows.keyedTumblingWindow(
              (String kv) -> kv, Duration.ofSeconds(1),
              () -> 0L, (m, prevCount) -> prevCount + 1,
              new StringSerde(), new LongSerde()
          )
              .setEarlyTrigger(Triggers.repeat(Triggers.count(1)))
              .setAccumulationMode(AccumulationMode.ACCUMULATING), "count")
          .map(windowPane -> {
            String word = windowPane.getKey().getKey();
            long count = windowPane.getMessage();
            return KV.of(word, count);
          })
          .sendTo(counts);
    }
  }
}

Spark

Spark structured streaming can create streams for Kafka by specifying a Kafka format.  Spark only supports writing strings or byte arrays to Kafka, so we use a Spark UDF to convert the count from a Long to a byte[].

public class SparkWordCount implements WordCount, java.io.Serializable {

  private StreamingQuery query;

  @Override
  public void countWords(
      String bootstrapServers, String zookeeperConnect,
      String inputTopic, String outputTopic) {

    SparkSession spark = SparkSession
        .builder()
        .appName("JavaStructuredKafkaWordCount")
        .config("spark.master", "local[*]")
        .getOrCreate();

    spark.udf().register(
        "serializeLong", new SparkLongSerializer(), DataTypes.BinaryType);

    // Create DataSet representing the stream of input lines from kafka
    Dataset<String> lines = spark
        .readStream()
        .format("kafka")
        .option("kafka.bootstrap.servers", bootstrapServers)
        .option("subscribe", inputTopic)
        .option("startingOffsets", "earliest")
        .load()
        .selectExpr("CAST(value as STRING)")
        .as(Encoders.STRING());

    // Generate running word count
    Dataset<Row> wordCounts = lines.flatMap(
        (FlatMapFunction<String, String>) x ->
            Arrays.asList(x.toLowerCase().split("\\W+"))
                .iterator(),
        Encoders.STRING()
    ).groupBy("value").count();

    // Start running the query that outputs the running counts to Kafka
    query = wordCounts
        .selectExpr("CAST(value AS STRING) key",
            "CAST(serializeLong(count) AS BINARY) value")
        .writeStream()
        .outputMode("update")
        .format("kafka")
        .option("kafka.bootstrap.servers", bootstrapServers)
        .option("topic", outputTopic)
        .option("checkpointLocation", Utils.tempDirectory().getAbsolutePath())
        .start();
  }

  @Override
  public void close() {
    query.stop();
  }

  static class SparkLongSerializer implements UDF1<Long, byte[]> {
    @Override
    public byte[] call(Long i) {
      return new LongSerializer().serialize(null, i.longValue());
    }
  }
}

Storm

In Storm we use a Kafka spout to read data from Kafka and then a Kafka bolt to write data.

public class StormWordCount implements WordCount {

  private LocalCluster cluster;

  @Override
  public void countWords(
      String bootstrapServers, String zookeeperConnect,
      String inputTopic, String outputTopic) {

    KafkaSpoutConfig<String, String> config =
        KafkaSpoutConfig.builder(bootstrapServers, inputTopic)
            .setRecordTranslator((r) ->
                new Values(r.value()), new Fields("value"))
            .build();
    KafkaSpout<String, String> spout = new KafkaSpout<>(config);

    TopologyBuilder builder = new TopologyBuilder();
    builder.setSpout("spout", spout);
    builder.setBolt("split", new SplitSentence(), 4).shuffleGrouping("spout");
    builder.setBolt("count", new WordCount(), 4)
        .fieldsGrouping("split", new Fields("word"));

    Properties props = new Properties();
    props.put("bootstrap.servers", bootstrapServers);
    props.put("acks", "1");
    props.put("key.serializer", StringSerializer.class.getName());
    props.put("value.serializer", LongSerializer.class.getName());

    KafkaBolt<String, Long> bolt = new KafkaBolt<String, Long>()
        .withProducerProperties(props)
        .withTopicSelector(new DefaultTopicSelector(outputTopic))
        .withTupleToKafkaMapper(
            new FieldNameBasedTupleToKafkaMapper<>("word","count"));
    builder.setBolt("forwardToKafka", bolt, 8)
        .fieldsGrouping("count", new Fields("word"));

    Config conf = new Config();
    cluster = new LocalCluster();
    cluster.submitTopology("wordCount", conf, builder.createTopology());
  }

  @Override
  public void close() {
    cluster.shutdown();
  }

  static class SplitSentence extends BaseBasicBolt {
    @Override
    public void execute(Tuple tuple, BasicOutputCollector collector) {
      String sentence = tuple.getString(0);

      for (String word : sentence.toLowerCase().split("\\W+")) {
        collector.emit(new Values(word, 1L));
      }
    }

    @Override
    public void declareOutputFields(OutputFieldsDeclarer declarer) {
      declarer.declare(new Fields("word", "count"));
    }
  }

  static class WordCount extends BaseBasicBolt {
    Map<String, Long> counts = new HashMap<>();

    @Override
    public void execute(Tuple tuple, BasicOutputCollector collector) {
      String word = tuple.getString(0);
      Long count = counts.get(word);
      if (count == null) {
        count = 0L;
      }
      count++;
      counts.put(word, count);
      collector.emit(new Values(word, count));
    }

    @Override
    public void declareOutputFields(OutputFieldsDeclarer declarer) {
      declarer.declare(new Fields("word", "count"));
    }
  }
}

Trident

Trident also uses a Kafka spout to read data and then a Kafka state factory to write data.

public class TridentWordCount implements WordCount {

  private LocalCluster cluster;

  @Override
  public void countWords(
      String bootstrapServers, String zookeeperConnect,
      String inputTopic, String outputTopic) {

    KafkaSpoutConfig<String, String> config =
        KafkaSpoutConfig.builder(bootstrapServers, inputTopic).build();
    KafkaTridentSpoutOpaque<String, String> spout =
        new KafkaTridentSpoutOpaque<>(config);

    TridentTopology topology = new TridentTopology();
    Stream wordCounts = topology.newStream("spout1", spout)
        .each(new Fields("value"), new Split(), new Fields("word"))
        .groupBy(new Fields("word"))
        .persistentAggregate(
            new MemoryMapState.Factory(), new Count(), new Fields("count"))
        .newValuesStream();

    Properties props = new Properties();
    props.put("bootstrap.servers", bootstrapServers);
    props.put("acks", "1");
    props.put("key.serializer", StringSerializer.class.getName());
    props.put("value.serializer", LongSerializer.class.getName());

    TridentKafkaStateFactory<String, Long> stateFactory =
        new TridentKafkaStateFactory<String, Long>()
            .withProducerProperties(props)
            .withKafkaTopicSelector(new DefaultTopicSelector(outputTopic))
            .withTridentTupleToKafkaMapper(
                new FieldNameBasedTupleToKafkaMapper<String, Long>(
                    "word","count"));
    wordCounts.partitionPersist(
        stateFactory,
        new Fields("word", "count"),
        new TridentKafkaStateUpdater(),
        new Fields()
    );

    Config conf = new Config();
    cluster = new LocalCluster();
    cluster.submitTopology("wordCount", conf, topology.build());
  }

  @Override
  public void close() {
    cluster.shutdown();
  }

  static class Split extends BaseFunction {
    @Override
    public void execute(TridentTuple tuple, TridentCollector collector) {
      String sentence = tuple.getString(0);
      for (String word : sentence.toLowerCase().split("\\W+")) {
        collector.emit(new Values(word));
      }
    }
  }
}
  1. Once KSQL supports an EXPLODE function, I can add a WordCount example for KSQL to the project.
Image

Leave a Reply