How to find distinct values in a stream of events

Problem:

you have events in a Kafka topic, and you want to filter out duplicate events based on a field in the event, producing a new stream of only unique events

Edit this page

Example use case:

Consider a topic with events that represent clicks on a website. Each event contains an IP address, a URL, and a timestamp. In this tutorial, we'll write a program that filters click events by the IP address within a window of time.

Code example:

Try it

1
Initialize the project

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

mkdir distinct-events && cd distinct-events

2
Get Confluent Platform

Next, create the following docker-compose.yml file to configure an instance of the Confluent Platform (on macOS you can paste directly to a file from the copy buffer with pbpaste > docker-compose.yml):

---
version: '2'

services:
  zookeeper:
    image: confluentinc/cp-zookeeper:5.3.1
    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.1
    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.1
    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 -d

3
Configure the project

We’ll use Gradle as our build system. Create a file named build.gradle with the following contents (on macOS pbpaste > build.gradle after copying the below):

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.3.0"
    implementation "io.confluent:kafka-streams-avro-serde:5.3.1"
    testCompile "org.apache.kafka:kafka-streams-test-utils:2.3.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.FindDistinctEvents"
    )
  }
}

shadowJar {
    archiveName = "kstreams-find-distinct-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 configuration file for development at configuration/dev.properties:

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

input.topic.name=clicks
input.topic.partitions=1
input.topic.replication.factor=1

output.topic.name=distinct-clicks
output.topic.partitions=1
output.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/click.avsc for the click events (on macOS, copy the text then execute in the terminal: pbpaste > src/main/avro/click.avsc):

{
  "namespace": "io.confluent.developer.avro",
  "type": "record",
  "name": "Click",
  "fields": [
    {"name": "ip", "type": "string"},
    {"name": "url", "type": "string"},
    {"name": "timestamp", "type": "string"}
  ]
}

Because this Avro schema is used in the Java code, it needs 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/FindDistinctEvents.java

Focusing on the buildTopology method, note how the Kafka Streams topology relies on a Transformer and a Window Store to filter out the duplicate IP addresses. Events are de-duped within a 2 minute window, and unique clicks are produced to a new topic named distinct-clicks.

package io.confluent.developer;

import io.confluent.developer.avro.Click;
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.*;
import org.apache.kafka.streams.kstream.Consumed;

import java.io.FileInputStream;
import java.io.IOException;
import java.time.Duration;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Properties;
import java.util.concurrent.CountDownLatch;

import io.confluent.kafka.serializers.AbstractKafkaAvroSerDeConfig;
import io.confluent.kafka.streams.serdes.avro.SpecificAvroSerde;
import org.apache.kafka.streams.kstream.KeyValueMapper;
import org.apache.kafka.streams.kstream.Produced;
import org.apache.kafka.streams.kstream.Transformer;
import org.apache.kafka.streams.processor.ProcessorContext;
import org.apache.kafka.streams.state.StoreBuilder;
import org.apache.kafka.streams.state.Stores;
import org.apache.kafka.streams.state.WindowStore;
import org.apache.kafka.streams.state.WindowStoreIterator;

public class FindDistinctEvents {

  private static final String storeName = "eventId-store";

  /**
   * Discards duplicate click events from the input stream by ip address
   * <p>
   * Duplicate records are detected based on ip address
   * The transformer remembers known ip addresses within an associated window state
   * store, which automatically purges/expires IPs from the store after a certain amount of
   * time has passed to prevent the store from growing indefinitely.
   * <p>
   * Note: This code is for demonstration purposes and was not tested for production usage.
   */
  private static class DeduplicationTransformer<K, V, E> implements Transformer<K, V, KeyValue<K, V>> {

    private ProcessorContext context;

    /**
     * Key: ip address
     * Value: timestamp (event-time) of the corresponding event when the event ID was seen for the
     * first time
     */
    private WindowStore<E, Long> eventIdStore;

    private final long leftDurationMs;
    private final long rightDurationMs;

    private final KeyValueMapper<K, V, E> idExtractor;

    /**
     * @param maintainDurationPerEventInMs how long to "remember" a known ip address
     *                                     during the time of which any incoming duplicates
     *                                     will be dropped, thereby de-duplicating the
     *                                     input.
     * @param idExtractor                  extracts a unique identifier from a record by which we de-duplicate input
     *                                     records; if it returns null, the record will not be considered for
     *                                     de-duping but forwarded as-is.
     */
    DeduplicationTransformer(final long maintainDurationPerEventInMs, final KeyValueMapper<K, V, E> idExtractor) {
      if (maintainDurationPerEventInMs < 1) {
        throw new IllegalArgumentException("maintain duration per event must be >= 1");
      }
      leftDurationMs = maintainDurationPerEventInMs / 2;
      rightDurationMs = maintainDurationPerEventInMs - leftDurationMs;
      this.idExtractor = idExtractor;
    }

    @Override
    @SuppressWarnings("unchecked")
    public void init(final ProcessorContext context) {
      this.context = context;
      eventIdStore = (WindowStore<E, Long>) context.getStateStore(storeName);
    }

    public KeyValue<K, V> transform(final K key, final V value) {
      final E eventId = idExtractor.apply(key, value);
      if (eventId == null) {
        return KeyValue.pair(key, value);
      } else {
        final KeyValue<K, V> output;
        if (isDuplicate(eventId)) {
          output = null;
          updateTimestampOfExistingEventToPreventExpiry(eventId, context.timestamp());
        } else {
          output = KeyValue.pair(key, value);
          rememberNewEvent(eventId, context.timestamp());
        }
        return output;
      }
    }

    private boolean isDuplicate(final E eventId) {
      final long eventTime = context.timestamp();
      final WindowStoreIterator<Long> timeIterator = eventIdStore.fetch(
              eventId,
              eventTime - leftDurationMs,
              eventTime + rightDurationMs);
      final boolean isDuplicate = timeIterator.hasNext();
      timeIterator.close();
      return isDuplicate;
    }

    private void updateTimestampOfExistingEventToPreventExpiry(final E eventId, final long newTimestamp) {
      eventIdStore.put(eventId, newTimestamp, newTimestamp);
    }

    private void rememberNewEvent(final E eventId, final long timestamp) {
      eventIdStore.put(eventId, timestamp, timestamp);
    }

    @Override
    public void close() {
      // Note: The store should NOT be closed manually here via `eventIdStore.close()`!
      // The Kafka Streams API will automatically close stores when necessary.
    }

  }

  private SpecificAvroSerde<Click> buildClicksSerde(final Properties envProps) {
    final SpecificAvroSerde<Click> serde = new SpecificAvroSerde<>();
    Map<String, String> config = new HashMap<>();
    config.put(AbstractKafkaAvroSerDeConfig.SCHEMA_REGISTRY_URL_CONFIG, envProps.getProperty("schema.registry.url"));
    serde.configure(config, false);
    return serde;
  }

  public Topology buildTopology(Properties envProps,
                                final SpecificAvroSerde<Click> clicksSerde) {
    final StreamsBuilder builder = new StreamsBuilder();

    final String inputTopic = envProps.getProperty("input.topic.name");
    final String outputTopic = envProps.getProperty("output.topic.name");

    // How long we "remember" an event.  During this time, any incoming duplicates of the event
    // will be, well, dropped, thereby de-duplicating the input data.
    //
    // The actual value depends on your use case.  To reduce memory and disk usage, you could
    // decrease the size to purge old windows more frequently at the cost of potentially missing out
    // on de-duplicating late-arriving records.
    final Duration windowSize = Duration.ofMinutes(2);

    // retention period must be at least window size -- for this use case, we don't need a longer retention period
    // and thus just use the window size as retention time
    final Duration retentionPeriod = windowSize;

    final StoreBuilder<WindowStore<String, Long>> dedupStoreBuilder = Stores.windowStoreBuilder(
            Stores.persistentWindowStore(storeName,
                    retentionPeriod,
                    windowSize,
                    false
            ),
            Serdes.String(),
            Serdes.Long());

    builder.addStateStore(dedupStoreBuilder);

    builder
      .stream(inputTopic, Consumed.with(Serdes.String(), clicksSerde))
      .transform( () -> new DeduplicationTransformer<>(windowSize.toMillis(), (key, value) -> value.getIp()), storeName)
      .to(outputTopic, Produced.with(Serdes.String(), clicksSerde));

    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.topic.name"),
        Integer.parseInt(envProps.getProperty("output.topic.partitions")),
        Short.parseShort(envProps.getProperty("output.topic.replication.factor"))));

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

  public static 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, Serdes.String().getClass());
    props.put(AbstractKafkaAvroSerDeConfig.SCHEMA_REGISTRY_URL_CONFIG, envProps.getProperty("schema.registry.url"));
    props.put(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, 0);

    return props;
  }
  public static 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 IOException {
    if (args.length < 1) {
      throw new IllegalArgumentException(
          "This program takes one argument: the path to an environment configuration file.");
    }

    new FindDistinctEvents().runRecipe(args[0]);
  }

  private void runRecipe(final String configPath) throws IOException {
    Properties envProps = this.loadEnvProperties(configPath);
    Properties streamProps = this.buildStreamsProperties(envProps);

    Topology topology = this.buildTopology(envProps, this.buildClicksSerde(envProps));
    this.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 (Ctrl-c). Run the application in a separate terminal in order to see any output as well as continue with the remaining instructions in the current terminal. The Kafka Streams application should produce a log entry similar to this State transition from REBALANCING to RUNNING to indicate it’s functioning correctly.

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

7
Produce sample click events to the input topic

In a new terminal, run:

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

When the console producer starts it may log some informational messages, and then it will pause, waiting to read input from the terminal. Below are sample events you can paste into the terminal one at a time. Press enter to send each record after pasting. (Note: if the client encounters a SerializationException, that’s likely because a stray newline was included in the pasted content.)

In the next steps we will run a consumer to observe the distinct click events. You can experiment with various orderings of the records in order to observe what makes a click event distinct. By default the distinct event window store looks for distinct clicks over a 2-minute duration.

{"ip":"10.0.0.1","url":"https://docs.confluent.io/current/tutorials/examples/kubernetes/gke-base/docs/index.html","timestamp":"2019-09-16T14:53:43+00:00"}
{"ip":"10.0.0.2","url":"https://www.confluent.io/hub/confluentinc/kafka-connect-datagen","timestamp":"2019-09-16T14:53:43+00:01"}
{"ip":"10.0.0.3","url":"https://www.confluent.io/hub/confluentinc/kafka-connect-datagen","timestamp":"2019-09-16T14:53:43+00:03"}
{"ip":"10.0.0.1","url":"https://docs.confluent.io/current/tutorials/examples/kubernetes/gke-base/docs/index.html","timestamp":"2019-09-16T14:53:43+00:00"}
{"ip":"10.0.0.2","url":"https://www.confluent.io/hub/confluentinc/kafka-connect-datagen","timestamp":"2019-09-16T14:53:43+00:01"}
{"ip":"10.0.0.3","url":"https://www.confluent.io/hub/confluentinc/kafka-connect-datagen","timestamp":"2019-09-16T14:53:43+00:03"}

8
Consume distinct events from the output topic

Leave your previous terminal running and open a new one that will run a consumer to view the distinct click events:

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

Depending on the cadence and values you produce in the steps above, you should see messages similar to the following:

{"ip":"10.0.0.1","url":"https://docs.confluent.io/current/tutorials/examples/kubernetes/gke-base/docs/index.html","timestamp":"2019-09-16T14:53:43+00:00"}
{"ip":"10.0.0.2","url":"https://www.confluent.io/hub/confluentinc/kafka-connect-datagen","timestamp":"2019-09-16T14:53:43+00:01"}
{"ip":"10.0.0.3","url":"https://www.confluent.io/hub/confluentinc/kafka-connect-datagen","timestamp":"2019-09-16T14:53:43+00:03"}

Test it

1
Create a test configuration file

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

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

input.topic.name=clicks
input.topic.partitions=1
input.topic.replication.factor=1

output.topic.name=distinct-clicks
output.topic.partitions=1
output.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/FindDistinctEventsTest.java. Testing a Kafka Streams application requires a bit of test harness code, but happily the org.apache.kafka.streams.TopologyTestDriver class makes this much more pleasant than it would otherwise be.

There is a test method in FindDistinctEventsTest annotated with @Test: shouldFilterDistinctEvents() which follows the common Arrange Act Assert (AAA) pattern. This is a simple method that runs our Streams topology using the TopologyTestDriver and some mocked data that is set up inside the test method.

package io.confluent.developer;

import io.confluent.developer.avro.Click;
import org.apache.avro.Schema;

import java.io.File;
import java.io.IOException;
import java.nio.file.Files;
import java.nio.file.Path;
import java.nio.file.Paths;
import java.util.*;

import io.confluent.kafka.schemaregistry.client.MockSchemaRegistryClient;
import io.confluent.kafka.schemaregistry.client.rest.exceptions.RestClientException;
import io.confluent.kafka.streams.serdes.avro.SpecificAvroSerde;
import org.apache.kafka.clients.producer.ProducerRecord;
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.StreamsConfig;
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.Before;
import org.junit.Test;

import static java.util.Arrays.asList;

public class FindDistinctEventsTest {

  private final static String TEST_CONFIG_FILE = "configuration/test.properties";
  private final static Path STATE_DIR =
          Paths.get(System.getProperty("user.dir"), "build");

  private Properties envProps;
  private Properties streamProps;

  public FindDistinctEventsTest() throws IOException {
    envProps = FindDistinctEvents.loadEnvProperties(TEST_CONFIG_FILE);
    streamProps = FindDistinctEvents.buildStreamsProperties(envProps);
    streamProps.put(StreamsConfig.STATE_DIR_CONFIG, STATE_DIR.toString());
  }

  private static SpecificAvroSerde<Click> makeSerializer(Properties envProps)
      throws IOException, RestClientException {

    final MockSchemaRegistryClient client = new MockSchemaRegistryClient();
    String inputTopic = envProps.getProperty("input.topic.name");
    String outputTopic = envProps.getProperty("output.topic.name");

    final Schema schema = Click.SCHEMA$;
    client.register(inputTopic + "-value", schema);
    client.register(outputTopic + "-value", schema);

    SpecificAvroSerde<Click> serde = new SpecificAvroSerde<>(client);

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

    return serde;
  }
  void deleteDirectory(Path path) throws IOException {
    Files.walk(path)
            .sorted(Comparator.reverseOrder())
            .map(Path::toFile)
            .forEach(File::delete);
  }
  @Before
  public void before() {
    try {
      deleteDirectory(Paths.get(
              STATE_DIR.toString(),
              envProps.getProperty("application.id")));
    } catch (IOException e) {
    }
  }

  @Test
  public void shouldFilterDistinctEvents() throws IOException, RestClientException {

    final FindDistinctEvents distinctifier  = new FindDistinctEvents();

    String inputTopic = envProps.getProperty("input.topic.name");
    String outputTopic = envProps.getProperty("output.topic.name");

    final SpecificAvroSerde<Click> clickSerde = makeSerializer(envProps);

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

    Serializer<String> keySerializer = Serdes.String().serializer();

    ConsumerRecordFactory<String, Click> inputFactory = new ConsumerRecordFactory<>(
            keySerializer, clickSerde.serializer());

    final List<Click> clicks = asList(
            new Click("10.0.0.1",
                    "https://docs.confluent.io/current/tutorials/examples/kubernetes/gke-base/docs/index.html",
            "2019-09-16T14:53:43+00:00"),
            new Click("10.0.0.2",
                    "https://www.confluent.io/hub/confluentinc/kafka-connect-datagen",
            "2019-09-16T14:53:43+00:01"),
            new Click("10.0.0.3",
            "https://www.confluent.io/hub/confluentinc/kafka-connect-datagen",
            "2019-09-16T14:53:43+00:03"),
            new Click("10.0.0.1",
            "https://docs.confluent.io/current/tutorials/examples/kubernetes/gke-base/docs/index.html",
            "2019-09-16T14:53:43+00:00"),
            new Click("10.0.0.2",
            "https://www.confluent.io/hub/confluentinc/kafka-connect-datagen",
            "2019-09-16T14:53:43+00:01"),
            new Click("10.0.0.3",
            "https://www.confluent.io/hub/confluentinc/kafka-connect-datagen",
            "2019-09-16T14:53:43+00:03"));

    final List<Click> expectedOutput = asList(clicks.get(0),clicks.get(1),clicks.get(2));

    for (Click clk : clicks) {
      testDriver.pipeInput(inputFactory.create(inputTopic, clk.getIp(), clk));
    }

    Deserializer<String> keyDeserializer = Serdes.String().deserializer();
    List<Click> actualOutput = new ArrayList<>();
    while (true) {
      ProducerRecord<String, Click>
          record =
          testDriver.readOutput(outputTopic, keyDeserializer, clickSerde.deserializer());

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

    Assert.assertEquals(expectedOutput, actualOutput);
  }
}

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=find-distinct-app
bootstrap.servers=<< FILL ME IN >>
schema.registry.url=<< FILL ME IN >>

input.topic.name=clicks
input.topic.partitions=<< FILL ME IN >>
input.topic.replication.factor=<< FILL ME IN >>

output.topic.name=distinct-clicks
output.topic.partitions=<< FILL ME IN >>
output.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-find-distinct: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-find-distinct:0.0.1 config.properties