...
Please keep the discussion on the mailing list rather than commenting on the wiki (wiki discussions get unwieldy fast).
Motivation
THIS STILL DRAFT:IS STILL WORK IN PROGRESS.
This KIP is inspired by the Discussion in KIP-456: Helper classes to make it simpler to write test logic with TopologyTestDriver
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 wrapping existing functionalityto 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
Code Block |
---|
language | java |
---|
title | TestInputTopicKafkaClient |
---|
|
package org.apache.kafka.streams.testclients;
//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();
} |
Code Block |
---|
|
package org.apache.kafka.clients.producer;
public class ProducerRecord<K, V> implements ClientRecord<K, V> {
... |
Code Block |
---|
language | java |
---|
title | TopologyTestDriver |
---|
|
package org.apache.kafka.streams;
public class TopologyTestDriver {
...
@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
// methods for result verification
@Deprecate public ProducerRecord<byte[], byte[]> readOutput(String topic);
@Deprecate public <K, V> ProducerRecord<K, V> readOutput(String topic, Deserializer<K> keyDeserializer, Deserializer<V> valueDeserializer);
...
} |
Code Block |
---|
language | java |
---|
title | TopologyTestDriver (new) |
---|
|
package org.apache.kafka.streams.test;
public class TopologyTestDriver {
public TopologyTestDriver(Topology topology, Properties config); // initialized WallClockMockTimeMs with System.currentTimeMillis()
public TopologyTestDriver(Topology topology, Properties config, long initialWallClockTimeMs);
// 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(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.
//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);
public final <K, V> TestInputTopic(final TopologyTestDriver driver, 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();
}
|
Code Block |
---|
language | java |
---|
title | TestInputTopic |
---|
|
package org.apache.kafka.streams.test;
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.
void configureTiming(, 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 |
---|
language | java |
---|
title | TestOutputTopic |
---|
|
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);Create by TopologyTestDriver, Constructors are package private
//Methods to readOutput
ProducerRecord<KClientRecord<K, V> readRecord();
KeyValue<K, V> readKeyValue();
V readValue();
//Output as collection
Map<K, V> readKeyValuesToMap();
List<KeyValue<K, V>> readKeyValuesToList();
List<V> readValuesToList();
} |
...