Class TopologyTestDriver

java.lang.Object
org.apache.kafka.streams.TopologyTestDriver
All Implemented Interfaces:
Closeable, AutoCloseable

public class TopologyTestDriver extends Object implements Closeable
This class makes it easier to write tests to verify the behavior of topologies created with Topology or StreamsBuilder. You can test simple topologies that have a single processor, or very complex topologies that have multiple sources, processors, sinks, or sub-topologies. Best of all, the class works without a real Kafka broker, so the tests execute very quickly with very little overhead.

Using the TopologyTestDriver in tests is easy: simply instantiate the driver and provide a Topology (cf. StreamsBuilder.build()) and config, create and use a TestInputTopic to supply an input records to the topology, and then create and use a TestOutputTopic to read and verify any output records by the topology.

Although the driver doesn't use a real Kafka broker, it does simulate Kafka consumers and producers that read and write raw byte[] messages. You can let TestInputTopic and TestOutputTopic to handle conversion form regular Java objects to raw bytes.

Driver setup

In order to create a TopologyTestDriver instance, you need a Topology and a config. The configuration needs to be representative of what you'd supply to the real topology, so that means including several key properties (cf. StreamsConfig). For example, the following code fragment creates a configuration that specifies a timestamp extractor, default serializers and deserializers for string keys and values:

 Properties props = new Properties();
 props.setProperty(StreamsConfig.DEFAULT_TIMESTAMP_EXTRACTOR_CLASS_CONFIG, CustomTimestampExtractor.class.getName());
 props.setProperty(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.String().getClass().getName());
 props.setProperty(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.String().getClass().getName());
 Topology topology = ...
 TopologyTestDriver driver = new TopologyTestDriver(topology, props);
 

Note that the TopologyTestDriver processes input records synchronously. This implies that commit.interval.ms and cache.max.bytes.buffering configuration have no effect. The driver behaves as if both configs would be set to zero, i.e., as if a "commit" (and thus "flush") would happen after each input record.

Processing messages

Your test can supply new input records on any of the topics that the topology's sources consume. This test driver simulates single-partitioned input topics. Here's an example of an input message on the topic named input-topic:


 TestInputTopic<String, String> inputTopic = driver.createInputTopic("input-topic", stringSerdeSerializer, stringSerializer);
 inputTopic.pipeInput("key1", "value1");
 
When TestInputTopic.pipeInput(Object, Object) is called, the driver passes the input message through to the appropriate source that consumes the named topic, and will invoke the processor(s) downstream of the source. If your topology's processors forward messages to sinks, your test can then consume these output messages to verify they match the expected outcome. For example, if our topology should have generated 2 messages on output-topic-1 and 1 message on output-topic-2, then our test can obtain these messages using the TestOutputTopic.readKeyValue() method:

 TestOutputTopic<String, String> outputTopic1 = driver.createOutputTopic("output-topic-1", stringDeserializer, stringDeserializer);
 TestOutputTopic<String, String> outputTopic2 = driver.createOutputTopic("output-topic-2", stringDeserializer, stringDeserializer);

 KeyValue<String, String> record1 = outputTopic1.readKeyValue();
 KeyValue<String, String> record2 = outputTopic2.readKeyValue();
 KeyValue<String, String> record3 = outputTopic1.readKeyValue();
 
Again, our example topology generates messages with string keys and values, so we supply our string deserializer instance for use on both the keys and values. Your test logic can then verify whether these output records are correct.

Note, that calling pipeInput() will also trigger event-time base punctuation callbacks. However, you won't trigger wall-clock type punctuations that you must trigger manually via advanceWallClockTime(Duration).

Finally, when completed, make sure your tests close() the driver to release all resources and processors.

Processor state

Some processors use Kafka state storage, so this driver class provides the generic getStateStore(String) as well as store-type specific methods so that your tests can check the underlying state store(s) used by your topology's processors. In our previous example, after we supplied a single input message and checked the three output messages, our test could also check the key value store to verify the processor correctly added, removed, or updated internal state. Or, our test might have pre-populated some state before submitting the input message, and verified afterward that the processor(s) correctly updated the state.

See Also: