Status
Current state: Under Discussion
Discussion thread:
JIRA:
Please keep the discussion on the mailing list rather than commenting on the wiki (wiki discussions get unwieldy fast).
Motivation
This KIP is inspired by the Discussion in KIP-456: Helper classes to make it simpler to write test logic with TopologyTestDriver
The stream application code is very compact and the test code is a lot of bigger code base than actual implementation of the application, that's why it would be good to get test code easily readable and understandable and that way also maintainable.
The proposal in KIP-456 was to add alternate way to input and output topic, but this KIP enhance those classes and deprecate old functionality to make clear interface for test writer to use.
When using the old TopologyTestDriver you need to call ConsumerRecordFactory to create ConsumerRecord passed into pipeInput method to write to topic. Also when calling readOutput to consume from topic, you need to provide correct Deserializers each time.
You easily end up writing helper methods in your test classes, but this can be avoided when adding generic input and output topic classes to implement the needed functionality.
Also the logic of the old TopologyTestDriver is confusing, when you need to pipe ConsumerRecords to produce record to input topic and receive ProducerRecords when consuming from output topic.
Non-existing topic and no record in the queue scenarious are modified to throw Exception instead of returning null.
Public Interfaces
package org.apache.kafka.clients; //New Interface to make possible to use ProducereRecord in TopologyTestDriver //ConsumerRecord not modified to implement this because timestamp() return long, not Long public interface ClientRecord<K, V> { String topic(); Headers headers(); K key(); V value(); Long timestamp(); }
package org.apache.kafka.clients.producer; public class ProducerRecord<K, V> implements ClientRecord<K, V> { ...
package org.apache.kafka.streams; public class TopologyTestDriver { public TopologyTestDriver(Topology topology, Properties config); // initialized WallClockMockTimeMs with System.currentTimeMillis() public TopologyTestDriver(Topology topology, Properties config, long initialWallClockTimeMs); //Deprecate old pipe and read methods @Deprecate public void pipeInput(ConsumerRecord<byte[], byte[]> record); // can trigger event-time punctuation @Deprecate public void pipeInput(List<ConsumerRecord<byte[], byte[]>> records); // can trigger event-time punctuation @Deprecate public ProducerRecord<byte[], byte[]> readOutput(String topic); @Deprecate public <K, V> ProducerRecord<K, V> readOutput(String topic, Deserializer<K> keyDeserializer, Deserializer<V> valueDeserializer); // event-time is automatically advanced based on the provided input records, and thus event-time punctuation are triggered automatically // wall-clock time is mocked and not advanced automatically; user can advance wall-clock time manually, and thus, trigger wall-clock time punctuations manually public void advanceWallClockTime(long advanceMs); // can trigger wall-clock-time punctuation // methods for TestTopic object creation //TODO investigate, is there way to find out serde based on topology public final <K, V> TestInputTopic<K, V> createInputTopic(final String topicName, final Serde<K> keySerde, final Serde<V> valueSerde); public final <K, V> TestOutputTopic<K, V> createOutputTopic(final String topicName, final Serde<K> keySerde, final Serde<V> valueSerde); public Map<String, StateStore> getAllStateStores() public StateStore getStateStore(String name); public <K, V> KeyValueStore<K, V> getKeyValueStore(String name); public <K, V> WindowStore<K, V> getWindowStore(String name); public <K, V> SessionStore<K, V> getSessionStore(String name); public void close(); }
package org.apache.kafka.streams; public class TestInputTopic<K, V> { //Create by TopologyTestDriver, Constructors are package private //Timestamp handling //Record timestamp can be provided when piping input or use internally tracked time initialized configured with configureTiming: //startTimestampMs the initial timestamp for generated records, if not provided uses current system time as start timestamp. //autoAdvanceMs the time increment per generated record, if not provided auto-advance is disabled. //Reinitialize timestamp and advanceMs public void configureTiming(final long startTimestampMs); public void configureTiming(final long startTimestampMs, final long autoAdvanceMs); //Advances the internally tracked time. void advanceTimeMs(final long advanceMs); //Methods to pipe single record void pipeInput(final V value); void pipeInput(final K key, final V value); // Use provided timestamp, does not auto advance internally tracked time. void pipeInput(final V value, final long timestampMs); void pipeInput(final K key, final V value, final long timestampMs); // Use record timestamp, does not auto advance internally tracked time. void pipeInput(final ClientRecord<K, V> record); //Methods to pipe list of records void pipeValueList(final List<V> values); void pipeKeyValueList(final List<KeyValue<K, V>> keyValues); // Use provided timestamp, does not auto advance internally tracked time. void pipeValueList(final List<V> values, final long startTimestamp, final long advanceMs); void pipeKeyValueList(final List<KeyValue<K, V>> keyValues, final long startTimestamp, final long advanceMs); // Use record timestamp, does not auto advance internally tracked time. void pipeRecordList(final List<? extends ClientRecord<K, V>> records); }
package org.apache.kafka.streams; public class TestOutputTopic<K, V> { //Create by TopologyTestDriver, Constructors are package private //Method to check queue size final long getQueueSize(); final boolean isEmpty(); //Methods to readOutput, throw NoSuchElement if no record in queue V readValue(); KeyValue<K, V> readKeyValue(); ClientRecord<K, V> readRecord(); //Output as collection List<V> readValuesToList(); Map<K, V> readKeyValuesToMap(); List<KeyValue<K, V>> readKeyValuesToList(); List<ClientRecord<K, V>> readRecordsToList(); }
package org.apache.kafka.streams.test; //OutputVerifier modified to accept ClientRecord instead of ProducerRecord public class OutputVerifier { public static <K,V> void compareValue(ClientRecord<K, V> record, V expectedValue) throws AssertionError; public static <K,V> void compareValue(ClientRecord<K, V> record, ClientRecord<K, V> expectedRecord) throws AssertionError; public static <K,V> void compareKeyValue(ClientRecord<K, V> record, K expectedKey, V expectedValue) throws AssertionError; public static <K,V> void compareKeyValue(ClientRecord<K, V> record, ClientRecord<K, V> expectedRecord) throws AssertionError; public static <K,V> void compareValueTimestamp(ClientRecord<K, V> record, V expectedValue, long expectedTimestamp) throws AssertionError; public static <K,V> void compareValueTimestamp(ClientRecord<K, V> record, ClientRecord<K, V> expectedRecord) throws AssertionError; public static <K,V> void compareKeyValueTimestamp(ClientRecord<K, V> record, K expectedKey, V expectedValue, long expectedTimestamp) throws AssertionError; public static <K,V> void compareKeyValueTimestamp(ClientRecord<K, V> record, ClientRecord<K, V> expectedRecord) throws AssertionError; }
package org.apache.kafka.streams.test; @Deprecated public class ConsumerRecordFactory<K, V> { ...
package org.apache.kafka.streams.test; //Mainly used by TestInputTopic public class TestRecordFactory<K, V> { // default public TestRecordFactory(final String topicName) // initialized startTimestampMs with System.currentTimeMillis() and autoAdvanceMs with zero public TestRecordFactory(final String topicName, long startTimestampMs); public TestRecordFactory(final String topicName, long startTimestampMs, long autoAdvanceMs); //Reinitialize timestamp and advanceMs public void configureTiming(final long startTimestampMs); public void configureTiming(final long startTimestampMs, final long autoAdvanceMs); //Advance current timestamp public void advanceTimeMs(final long advanceMs); // create single records with default topic name and given timestamp public TestRecord<K, V> create(K key, V value, long timestampMs); public TestRecord<K, V> create(V value, long timestampMs); public TestRecord<K, V> create(K key, V value, Headers headers, long timestampMs); public TestRecord<K, V> create(V value, Headers headers, long timestampMs); // create single records with default topic name and generated timestamp public TestRecord<K, V> create(K key, V value); public TestRecord<K, V> create(V value); public TestRecord<K, V> create(K key, V value, Headers headers); public TestRecord<K, V> create(V value, Headers headers); // create list of records with default topic name internally tracked time public List<TestRecord<K, V>> create(final List<KeyValue<K, V>> keyValues); // does not auto advance internally tracked time. public List<TestRecord<K, V>> create(final List<KeyValue<K, V>> keyValues, long startTimestamp, long advanceMs); public List<TestRecord<K, V>> create(final List<KeyValue<K, V>> keyValues, long startTimestamp); //advanceMs =1 }
package org.apache.kafka.streams.test; public class TestRecord<K, V> implements org.apache.kafka.clients.ClientRecord<K, V> { //Constructors public TestRecord(String topic, Long timestamp, K key, V value, Headers headers); public TestRecord(String topic, Long timestamp, K key, V value); public TestRecord(String topic, K key, V value, Headers headers); public TestRecord(String topic, K key, V value); public TestRecord(String topic, V value); //Constructor by based on existing record public TestRecord(ConsumerRecord<K, V> record); public TestRecord(ClientRecord<K, V> record); // ClientRecord interface public String topic(); public Headers headers(); public K key(); public V value(); public Long timestamp(); //Overrides public String toString(); public boolean equals(Object o); public int hashCode(); }
Proposed Changes
This improvement adds TestInputTopic class which replaces TopologyTestDriver and ConsumerRecordFactory methods as one class to be used to write to Input Topics and TestOutputTopic class which collects TopologyTestDriver reading methods and provide typesafe read methods.
public class SimpleTopicTest { private TopologyTestDriver testDriver; private TestInputTopic<String, String> inputTopic; private TestOutputTopic<String, String> outputTopic; @Before public void setup() { testDriver = new TopologyTestDriver(TestStream.getTopology(), TestStream.getConfig()); inputTopic = testDriver.createInputTopic(TestStream.INPUT_TOPIC, new Serdes.StringSerde(), new Serdes.StringSerde()); outputTopic = testDriver.createOutputTopic(TestStream.OUTPUT_TOPIC, new Serdes.StringSerde(), new Serdes.LongSerde()); } @After public void tearDown() { testDriver.close(); } @Test public void testOneWord() { //Feed word "Hello" to inputTopic and no kafka key, timestamp is irrelevant in this case inputTopic.pipeInput("Hello"); assertThat(outputTopic.readValue()).isEqualTo("Hello"); //No more output in topic assertThat(outputTopic.isEmpty()).isTrue(); } }
- New Example utilizing new classes test added to streams/examples/src/test/java/org/apache/kafka/streams/examples/wordcount/WordCountDemoTest.java
- Examples in Testing Kafka Streams https://kafka.apache.org/22/documentation/streams/developer-guide/testing.html updated to use new TopolocyTestDriver, TestInputTopic and TestOutputTopic
Compatibility, Deprecation, and Migration Plan
There are no compatibility issues.
The tests utilizing old TopologyTestDriver can still use deprecated methods.
Migration plan ???
Rejected Alternatives
- This is will replace KIP-456: Helper classes to make it simpler to write test logic with TopologyTestDriver if accepted
- Deprecate current TestTopologyDriver and move new to test package. This would have enabled to keep also TestInputTopic and TestOutputTopic classes in test package, not in very crowded streams root package.