THIS IS A TEST INSTANCE. ALL YOUR CHANGES WILL BE LOST!!!!
Table of Contents |
---|
Status
Current state: Under Discussion'Accepted'
Discussion thread: here
JIRA:
Jira | ||||||||
---|---|---|---|---|---|---|---|---|
|
...
Code Block | ||||
---|---|---|---|---|
| ||||
package org.apache.kafka.clientsstreams; //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> {public class TopologyTestDriver { public TopologyTestDriver(Topology topology, Properties config); // existing constructor @Deprecate public TopologyTestDriver(Topology topology, Properties config, long initialWallClockTimeMs); Stringpublic topicTopologyTestDriver(); Topology topology, Properties config, HeadersInstant headers(initialWallClockTime); @Deprecate Kpublic void keyadvanceWallClockTime(long advanceMs); // can V value(); Long timestamp(); } |
Code Block | ||
---|---|---|
| ||
package org.apache.kafka.clients.producer;
public class ProducerRecord<K, V> implements ClientRecord<K, V> {
... |
Code Block | ||||
---|---|---|---|---|
| ||||
package org.apache.kafka.streams; public class TopologyTestDriver { public TopologyTestDriver(Topology topology, Properties configtrigger wall-clock-time punctuation public void advanceWallClockTime(Duration advance); // can trigger wall-clock-time punctuation //Deprecate old pipe and read methods @Deprecate public void pipeInput(ConsumerRecord<byte[], byte[]> record); // initializedcan WallClockMockTimeMstrigger with System.currentTimeMillis() public TopologyTestDriver(Topology topology, Properties config, long initialWallClockTimeMs); //Deprecate old pipe and read methodsevent-time punctuation @Deprecate public void pipeInput(List<ConsumerRecord<byte[], byte[]>> records); // can trigger event-time punctuation @Deprecate public void pipeInput(ConsumerRecord<byteProducerRecord<byte[], byte[]> recordreadOutput(String topic); // can trigger event-time punctuation @Deprecate public <K, void pipeInput(List<ConsumerRecord<byte[], byte[]>> records); // can trigger event-time punctuation @Deprecate public ProducerRecord<byte[], byte[]> readOutput(String topic); @Deprecate publicV> ProducerRecord<K, V> readOutput(String topic, Deserializer<K> keyDeserializer, Deserializer<V> valueDeserializer); // methods for TestTopic object creation public final <K, V> ProducerRecord<KTestOutputTopic<K, V> readOutputcreateOutputTopic(final String topictopicName, Deserializer<K>final keyDeserializerSerializer<K> keySerializer, final Deserializer<V>Serializer<V> valueDeserializervalueSerializer); // event- Uses current system time isas automaticallystart 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(); } timestamp. Auto-advance is disabled. public final <K, V> TestInputTopic<K, V> createInputTopic(final String topicName, final Deserializer<K> keyDeserializer, final Deserializer<V> valueDeserializer); //Uses provided startTimestamp and autoAdvance duration for timestamp generation public final <K, V> TestInputTopic<K, V> createInputTopic(final String topicName, final Deserializer<K> keyDeserializer, final Deserializer<V> valueDeserializer, final Instant startTimestamp, final Duration autoAdvance); ... } |
Code Block | ||||
---|---|---|---|---|
| ||||
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 configured with parameters:
//startTimestamp the initial timestamp for generated records, if not provided uses current system time as start timestamp.
//autoAdvance the time increment per generated record, if not provided auto-advance is disabled.
//Advances the internally tracked time.
void advanceTime(final Duration advance);
//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 Instant timestamp);
void pipeInput(final K key, final V value, final Instant timestamp);
// Method with long provided to support easier migration of old tests
void pipeInput(final K key, final V value, final long timestampMs);
// If record timestamp set, does not auto advance internally tracked time. | ||||
Code Block | ||||
| ||||
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 keyTestRecord<K, finalV> V valuerecord); //Methods Useto pipe providedlist timestamp,of doesrecords not auto advance internallyvoid tracked time. void pipeInput(final V value, final long timestampMspipeValueList(final List<V> values); void pipeInputpipeKeyValueList(final K keyList<KeyValue<K, 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); } |
Code Block | ||||
---|---|---|---|---|
| ||||
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();
} |
Code Block | ||||
---|---|---|---|---|
| ||||
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;
} |
Code Block | ||||
---|---|---|---|---|
| ||||
package org.apache.kafka.streams.test;
@Deprecated
public class ConsumerRecordFactory<K, V> {
... |
Code Block | ||||
---|---|---|---|---|
| ||||
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
} |
V>> keyValues);
// Use provided timestamp, does not auto advance internally tracked time.
void pipeValueList(final List<V> values, final Instant startTimestamp, final Duration advanceMs);
void pipeKeyValueList(final List<KeyValue<K, V>> keyValues, final Instant startTimestamp, final Duration advanceMs);
// If record timestamp set, does not auto advance internally tracked time.
void pipeRecordList(final List<? extends TestRecord<K, V>> records);
} |
Code Block | ||||
---|---|---|---|---|
| ||||
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();
TestRecord<K, V> readRecord();
//Output as collection
List<V> readValuesToList();
List<KeyValue<K, V>> readKeyValuesToList();
Map<K, V> readKeyValuesToMap();
List<TestRecord<K, V>> readRecordsToList();
} |
Code Block | ||||
---|---|---|---|---|
| ||||
package org.apache.kafka.streams.test;
public class TestRecord<K, V> {
//Constructors
public TestRecord(final V value);
public TestRecord(final K key, final V value);
public TestRecord(final K key, final V value, final Headers headers);
public TestRecord(final K key, final V value, final Instant recordTime);
public TestRecord(final K key, final V value, final Headers headers, final Instant recordTime);
public TestRecord(final K key, final V value, final Headers headers, final Long timestamp);
//Constructor based on existing record
public TestRecord(final ConsumerRecord<K, V> record | ||||
Code Block | ||||
| ||||
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(Stringfinal topicProducerRecord<K, Long timestamp, K key, V valueV> record); // Methods like in publicProducerRecord TestRecord(String topic, K key, V value,/ ConsumerRecord public Headers headers(); public K TestRecord(String topic, K key,key(); public V value(); public TestRecord(String topic, V valueLong timestamp(); //Constructor Getters by based onpublic existing recordHeaders getHeaders(); public TestRecord(ConsumerRecord<K, V> recordK getKey(); public TestRecord(ClientRecord<K, V> recordV getValue(); public Instant getRecordTime(); // ClientRecord interfaceOverrides public String toString(); public Stringboolean topicequals(Object o); public Headersint headershashCode(); } |
Code Block | ||||
---|---|---|---|---|
| ||||
package org.apache.kafka.streams.test; //Recommended to use normal publicassertion library methods @Deprecated public class OutputVerifier { ... |
Code Block | ||||
---|---|---|---|---|
| ||||
package org.apache.kafka.streams.test; //Similar functionality now in TestInputTopic @Deprecated public class ConsumerRecordFactory<K, V> { ...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.
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 = testDriver.createInputTopic(TestStream.INPUT_TOPIC, new Serdes.StringSerdeStringDeserializer(), new Serdes.StringSerdeStringDeserializer()); outputTopic = testDriver.createOutputTopic(TestStream.OUTPUT_TOPIC, new Serdes.StringSerdeStringSerializer(), new Serdes.LongSerdeLongSerializer()); } @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(); } } |
...
The tests utilizing old TopologyTestDriver can still use deprecated methods.Migration plan ???
Rejected Alternatives
- This is will replace replacing 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.
- Add ClientRecord interface to client package and modifiy ProducerRecord (and / or ConsumerRecord) to implement it, to be to utilize OutputVerifier with ProducerRecord and TestRecord