Table of Contents |
---|
This page is meant as a template for writing a KIP. To create a KIP choose Tools->Copy on this page and modify with your content and replace the heading with the next KIP number and a description of your issue. Replace anything in italics with your own description.
Status
Current state: KIP DRAFT [One of "Under Discussion", "Accepted", "Rejected"] Discarded in favor of KIP-470: TopologyTestDriver test input and output usability improvements
Discussion thread: here [Change the link from the KIP proposal email archive to your own email thread]
JIRA:
Jira | ||||||||
---|---|---|---|---|---|---|---|---|
|
...
Code Block | ||||
---|---|---|---|---|
| ||||
package org.apache.kafka.streams.test; public class TestInputTopic<K, V> { //Timestamp handling //Record timestamp can be provided when piping input or use internally tracked time initialized with constructors: //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. //TODO //Constructors with serializers TestInputTopic(final TopologyTestDriver driver, final String topicName, final Serializer<K> keySerializer, final Serializer<V> valueSerializer); TestInputTopic(final TopologyTestDriver driver, final String topicName, final Serializer<K> keySerializer, final Serializer<V> valueSerializer, final long startTimestampMs); TestInputTopic(final TopologyTestDriver driver, final String topicName, final Serializer<K> keySerializer, final Serializer<V> valueSerializer, final long startTimestampMs, final long autoAdvanceMs); //Constructors with serdes TestInputTopic(final TopologyTestDriver driver, final String topicName, final Serde<K> keySerde, final Serde<V> valueSerde); TestInputTopic(final TopologyTestDriver driver, final String topicName, final Serde<K> keySerde, final Serde<V> valueSerde, final long startTimestampMs); TestInputTopic(final TopologyTestDriver driver, final String topicName, final Serde<K> keySerde, final Serde<V> valueSerde, 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); void pipeInput(final V value, final long timestampMs); void pipeInput(final K key, final V value, final long timestampMs); void pipeInput(final K key, final V value, final Headers headers); void pipeInput(final K key, final V value, final Headers headers, final long timestampMs); //Methods to pipe list of records void pipeKeyValueList(final List<KeyValue<K, V>> keyValues); void pipeValueList(final List<V> values); void pipeKeyValueList(final List<KeyValue<K, V>> keyValues, final long startTimestamp, final long advanceMs); void pipeValueList(final List<V> values, final long startTimestamp, final long advanceMs); } |
Code Block | ||||
---|---|---|---|---|
| ||||
package org.apache.kafka.streams.test; public class TestOutputTopic<K, V> { //Constructor with serializers TestOutputTopic(final TopologyTestDriver driver, final String topic, final Serde<K> keySerde, final Serde<V> valueSerde); //Constructor with serdes TestOutputTopic(final TopologyTestDriver driver, final String topic, final Deserializer<K> keyDeserializer, final Deserializer<V> valueDeserializer); //TODO //Methods to readOutput ProducerRecord<K, V> readRecord(); KeyValue<K, V> readKeyValue(); V readValue(); //Output as collection Map<K, V> readKeyValuesToMap(); List<KeyValue<K, V>> readKeyValuesToList(); List<V> readValuesToList(); } |
Proposed Changes
This improvement adds TestInputTopic class which wraps 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.
Code Block | ||||
---|---|---|---|---|
| ||||
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 = new TestInputTopic<>(testDriver, TestStream.INPUT_TOPIC, new Serdes.StringSerde(), new Serdes.StringSerde());
outputTopic = new TestOutputTopic<>(testDriver, TestStream.OUTPUT_TOPIC, new Serdes.StringSerde(), new Serdes.StringSerde());
}
@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.readRecord()).isNull();
}
} |
- Examples in
- Examples ??
- New Classes need to updated to Testing Kafka Streams https://kafka.apache.org/22/documentation/streams/developer-guide/testing.html updated to use TestInputTopic and TestOutputTopic
Compatibility, Deprecation, and Migration Plan
We are This is only adding new classes. The tests utilizing directly TopologyTestDriver can still be used. There are no compatiblity issuescompatibility issues.
Rejected Alternatives
...
- It was considered to add methods to to return Iterable like in KIP-451: Make TopologyTestDriver output iterable , but it seems to be redundant with these List based methods